You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2020/06/30 13:53:15 UTC

[impala] branch master updated (930264a -> 2a48f7d)

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

stigahuang pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from 930264a  IMPALA-9515: Full ACID Milestone 3: Read support for "original files"
     new 931063f  IMPALA-9213: Add query retry info to GetLog result
     new 1fbca6d  IMPALA-9569: Fix progress bar and live_summary to show info of the retried query
     new 2a48f7d  IMPALA-9890 (Part 1): Add more TPCDS queries to Impala's test suite

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/service/impala-beeswax-server.cc            |  10 ++
 be/src/service/impala-hs2-server.cc                |  12 +-
 be/src/service/impala-server.cc                    |  38 ++++-
 be/src/service/impala-server.h                     |  10 ++
 shell/impala_client.py                             |  21 +++
 shell/impala_shell.py                              |  17 ++-
 testdata/datasets/tpcds/tpcds_schema_template.sql  |  19 +++
 .../tpcds/queries/tpcds-decimal_v2-q26.test        | 124 +++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q30.test        | 126 +++++++++++++++++
 .../queries/tpcds-decimal_v2-q31.test}             |  88 ++++++------
 .../tpcds/queries/tpcds-decimal_v2-q47.test        | 154 +++++++++++++++++++++
 .../queries/tpcds-decimal_v2-q48.test}             |  23 +--
 .../tpcds/queries/tpcds-decimal_v2-q57.test        | 151 ++++++++++++++++++++
 ...tpcds-query58.sql => tpcds-decimal_v2-q58.test} |  31 +++--
 .../tpcds/queries/tpcds-decimal_v2-q59.test        | 147 ++++++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q63.test        |  76 ++++++++++
 .../queries/tpcds-decimal_v2-q83.test}             |  47 ++++---
 .../queries/tpcds-decimal_v2-q85.test}             |  30 ++--
 .../tpcds/queries/tpcds-decimal_v2-q89.test        | 131 ++++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q26.test    | 124 +++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q30.test    | 126 +++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q47.test    | 154 +++++++++++++++++++++
 .../queries/tpcds-q48.test                         |  23 +--
 testdata/workloads/tpcds/queries/tpcds-q57.test    | 151 ++++++++++++++++++++
 .../{raw/tpcds-query58.sql => tpcds-q58.test}      |  31 +++--
 testdata/workloads/tpcds/queries/tpcds-q59.test    | 147 ++++++++++++++++++++
 .../{raw/tpcds-query63.sql => tpcds-q63.test}      |  61 +++++++-
 .../queries/tpcds-q83.test                         |  47 ++++---
 .../queries/tpcds-q85.test                         |  30 ++--
 testdata/workloads/tpcds/queries/tpcds-q89.test    | 131 ++++++++++++++++++
 tests/common/impala_connection.py                  |   5 +-
 tests/custom_cluster/test_query_retries.py         | 112 +++++++++++++--
 tests/query_test/test_tpcds_queries.py             |  69 +++++++++
 tests/util/parse_util.py                           |   2 +-
 34 files changed, 2266 insertions(+), 202 deletions(-)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q26.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q30.test
 copy testdata/workloads/{tpcds-unmodified/queries/tpcds-q31.test => tpcds/queries/tpcds-decimal_v2-q31.test} (93%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q47.test
 copy testdata/workloads/{tpcds-unmodified/queries/tpcds-q48.test => tpcds/queries/tpcds-decimal_v2-q48.test} (74%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q57.test
 copy testdata/workloads/tpcds/queries/{raw/tpcds-query58.sql => tpcds-decimal_v2-q58.test} (74%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q59.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q63.test
 copy testdata/workloads/{tpcds-unmodified/queries/tpcds-q83.test => tpcds/queries/tpcds-decimal_v2-q83.test} (55%)
 copy testdata/workloads/{tpcds-unmodified/queries/tpcds-q85.test => tpcds/queries/tpcds-decimal_v2-q85.test} (81%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-decimal_v2-q89.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q26.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q30.test
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q47.test
 copy testdata/workloads/{tpcds-unmodified => tpcds}/queries/tpcds-q48.test (74%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q57.test
 copy testdata/workloads/tpcds/queries/{raw/tpcds-query58.sql => tpcds-q58.test} (74%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q59.test
 copy testdata/workloads/tpcds/queries/{raw/tpcds-query63.sql => tpcds-q63.test} (51%)
 copy testdata/workloads/{tpcds-unmodified => tpcds}/queries/tpcds-q83.test (53%)
 copy testdata/workloads/{tpcds-unmodified => tpcds}/queries/tpcds-q85.test (82%)
 create mode 100644 testdata/workloads/tpcds/queries/tpcds-q89.test


[impala] 01/03: IMPALA-9213: Add query retry info to GetLog result

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 931063f0f2afc9cf6ea139b431db1a39f12e1f6e
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Thu Jun 18 11:50:55 2020 +0800

    IMPALA-9213: Add query retry info to GetLog result
    
    Beeswax clients use get_log() to retrieve the warning/error message
    after the query finishes. HS2 clients use GetLog() for the same purpose.
    This patch adds the retry information into the returned result if the
    query is retried. So clients that print the log can show the original
    query failure and the retried query id.
    
    This patch also modifies impala-shell to extract the retried query id
    and print the retried query link.
    
    Here's an example of the impala-shell output:
    
    Query: select count(*) from functional.alltypes where bool_col = sleep(60)
    Query submitted at: 2020-06-18 21:23:52 (Coordinator: http://quanlong-OptiPlex-BJ:25000)
    Query progress can be monitored at: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=7944ffee4d81cdd4:e7f9357a00000000
    +----------+
    | count(*) |
    +----------+
    | 3650     |
    +----------+
    WARNINGS: Original query failed:
    Failed due to unreachable impalad(s): quanlong-OptiPlex-BJ:22001
    
    Query has been retried using query id: 934b2734f67a1161:a0dbd60200000000
    Retried query link: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=934b2734f67a1161:a0dbd60200000000
    
    Tests:
     - Add tests in test_query_retries.py to verify client logs returned
       from GetLog().
     - Run test_query_retries.py.
     - Manually run queries in impala-shell and kill impalads. Verify
       printed messages when the retried queries succeed or fail.
    
    Change-Id: I58cf94f91a0b92eb9a3088bee3894ac157a954dc
    Reviewed-on: http://gerrit.cloudera.org:8080/16093
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/impala-beeswax-server.cc    | 10 ++++++
 be/src/service/impala-hs2-server.cc        | 12 ++++++-
 be/src/service/impala-server.cc            |  3 ++
 be/src/service/impala-server.h             |  2 ++
 shell/impala_client.py                     | 21 +++++++++++
 shell/impala_shell.py                      |  3 +-
 tests/custom_cluster/test_query_retries.py | 58 ++++++++++++++++++++++++------
 7 files changed, 96 insertions(+), 13 deletions(-)

diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index 8a80ca5..f1e8d35 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -320,6 +320,15 @@ void ImpalaServer::get_log(string& log, const LogContextId& context) {
       SQLSTATE_GENERAL_ERROR);
   stringstream error_log_ss;
 
+  if (query_handle->IsRetriedQuery()) {
+    QueryHandle original_query_handle;
+    RAISE_IF_ERROR(GetQueryHandle(query_id, &original_query_handle),
+        SQLSTATE_GENERAL_ERROR);
+    DCHECK(!original_query_handle->query_status().ok());
+    error_log_ss << Substitute(GET_LOG_QUERY_RETRY_INFO_FORMAT,
+        original_query_handle->query_status().GetDetail(),
+        PrintId(query_handle->query_id()));
+  }
   {
     // Take the lock to ensure that if the client sees a exec_state == ERROR, it is
     // guaranteed to see the error query_status.
@@ -343,6 +352,7 @@ void ImpalaServer::get_log(string& log, const LogContextId& context) {
     if (!coord_errors.empty()) error_log_ss << coord_errors << "\n";
   }
   log = error_log_ss.str();
+  VLOG_RPC << "get_log(): query_id=" << PrintId(query_id) << ", log=" << log;
 }
 
 void ImpalaServer::get_default_configuration(vector<ConfigVariable>& configurations,
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index 757c4e9..64e945b 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -931,7 +931,16 @@ void ImpalaServer::GetLog(TGetLogResp& return_val, const TGetLogReq& request) {
     // Report progress
     ss << coord->progress().ToString() << "\n";
   }
-  // Report the query status, if the query failed.
+  // Report the query status, if the query failed or has been retried.
+  if (query_handle->IsRetriedQuery()) {
+    QueryHandle original_query_handle;
+    HS2_RETURN_IF_ERROR(return_val, GetQueryHandle(query_id, &original_query_handle),
+        SQLSTATE_GENERAL_ERROR);
+    DCHECK(!original_query_handle->query_status().ok());
+    ss << Substitute(GET_LOG_QUERY_RETRY_INFO_FORMAT,
+        original_query_handle->query_status().GetDetail(),
+        PrintId(query_handle->query_id()));
+  }
   {
     // Take the lock to ensure that if the client sees a query_state == EXCEPTION, it is
     // guaranteed to see the error query_status.
@@ -966,6 +975,7 @@ void ImpalaServer::GetLog(TGetLogResp& return_val, const TGetLogReq& request) {
   }
   return_val.log = ss.str();
   return_val.status.__set_statusCode(thrift::TStatusCode::SUCCESS_STATUS);
+  VLOG_RPC << "GetLog(): query_id=" << PrintId(query_id) << ", log=" << return_val.log;
 }
 
 void ImpalaServer::GetExecSummary(TGetExecSummaryResp& return_val,
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 96a8857..ac5dfd6 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -362,6 +362,9 @@ const char* ImpalaServer::SQLSTATE_SYNTAX_ERROR_OR_ACCESS_VIOLATION = "42000";
 const char* ImpalaServer::SQLSTATE_GENERAL_ERROR = "HY000";
 const char* ImpalaServer::SQLSTATE_OPTIONAL_FEATURE_NOT_IMPLEMENTED = "HYC00";
 
+const char* ImpalaServer::GET_LOG_QUERY_RETRY_INFO_FORMAT =
+    "Original query failed:\n$0\nQuery has been retried using query id: $1\n";
+
 // Interval between checks for query expiration.
 const int64_t EXPIRATION_CHECK_INTERVAL_MS = 1000L;
 
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index 91dc3af..5fc66e8 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -645,6 +645,8 @@ class ImpalaServer : public ImpalaServiceIf,
   static const char* SQLSTATE_SYNTAX_ERROR_OR_ACCESS_VIOLATION;
   static const char* SQLSTATE_GENERAL_ERROR;
   static const char* SQLSTATE_OPTIONAL_FEATURE_NOT_IMPLEMENTED;
+  /// String format of retry information returned in GetLog() RPCs.
+  static const char* GET_LOG_QUERY_RETRY_INFO_FORMAT;
 
   /// Used in situations where the client provides a session ID and a query ID and the
   /// caller needs to validate that the query can be accessed from the session. The two
diff --git a/shell/impala_client.py b/shell/impala_client.py
index 8973d59..cb7cb39 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -141,6 +141,9 @@ class ImpalaClient(object):
     # query cancellation.
     self.is_query_cancelled = False
     self.verbose = verbose
+    # This is set in connect(). It's used in constructing the retried query link after
+    # we parse the retried query id.
+    self.webserver_address = None
 
   def connect(self):
     """Creates a connection to an Impalad instance. Returns a tuple with the impala
@@ -227,6 +230,10 @@ class ImpalaClient(object):
     'd74d8ce632c9d4d0:75c5a51100000000'"""
     raise NotImplementedError()
 
+  def get_query_link(self, query_id):
+    """Return the URL link to the debug page of the query"""
+    return "%s/query_plan?query_id=%s" % (self.webserver_address, query_id)
+
   def wait_to_finish(self, last_query_handle, periodic_callback=None):
     """Wait until the results can be fetched for 'last_query_handle' or until the
     query encounters an error or is cancelled. Raises an exception if the query
@@ -321,6 +328,16 @@ class ImpalaClient(object):
     (e.g. warnings)."""
     return self._get_warn_or_error_log(last_query_handle, False)
 
+  def _append_retried_query_link(self, get_log_result):
+    """Append the retried query link if the original query has been retried"""
+    query_id_search = re.search("Query has been retried using query id: (.*)\n",
+                                get_log_result)
+    if query_id_search and len(query_id_search.groups()) >= 1:
+      retried_query_id = query_id_search.group(1)
+      get_log_result += "Retried query link: %s" % \
+                        self.get_query_link(retried_query_id)
+    return get_log_result
+
   def _get_http_transport(self, connect_timeout_ms):
     """Creates a transport with HTTP as the base."""
     # Older python versions do not support SSLContext needed by ImpalaHttpClient. More
@@ -690,6 +707,7 @@ class ImpalaHS2Client(ImpalaClient):
     # PingImpalaHS2Service rpc is idempotent and so safe to retry.
     resp = self._do_hs2_rpc(PingImpalaHS2Service, retry_on_error=True)
     self._check_hs2_rpc_status(resp.status)
+    self.webserver_address = resp.webserver_address
     return (resp.version, resp.webserver_address)
 
   def _create_query_req(self, query_str, set_query_options):
@@ -891,6 +909,7 @@ class ImpalaHS2Client(ImpalaClient):
     # Strip progress message out of HS2 log.
     log = HS2_LOG_PROGRESS_REGEX.sub("", log)
     if log and log.strip():
+      log = self._append_retried_query_link(log)
       type_str = "WARNINGS" if warn is True else "ERROR"
       return "%s: %s" % (type_str, log)
     return ""
@@ -1037,6 +1056,7 @@ class ImpalaBeeswaxClient(ImpalaClient):
       raise
     except TTransportException as e:
       raise DisconnectedException("Error communicating with impalad: %s" % e)
+    self.webserver_address = resp.webserver_address
     return (resp.version, resp.webserver_address)
 
   def _create_query_req(self, query_str, set_query_options):
@@ -1144,6 +1164,7 @@ class ImpalaBeeswaxClient(ImpalaClient):
       type_str = "warn" if warn is True else "error"
       return "Failed to get %s log: %s" % (type_str, rpc_status)
     if log and log.strip():
+      log = self._append_retried_query_link(log)
       type_str = "WARNINGS" if warn is True else "ERROR"
       return "%s: %s" % (type_str, log)
     return ""
diff --git a/shell/impala_shell.py b/shell/impala_shell.py
index 244bef2..87436f5 100755
--- a/shell/impala_shell.py
+++ b/shell/impala_shell.py
@@ -1146,8 +1146,7 @@ class ImpalaShell(cmd.Cmd, object):
       self.last_summary = time.time()
       if print_web_link:
         self._print_if_verbose(
-            "Query progress can be monitored at: %s/query_plan?query_id=%s" %
-            (self.webserver_address,
+            "Query progress can be monitored at: %s" % self.imp_client.get_query_link(
              self.imp_client.get_query_id_str(self.last_query_handle)))
 
       wait_to_finish = self.imp_client.wait_to_finish(self.last_query_handle,
diff --git a/tests/custom_cluster/test_query_retries.py b/tests/custom_cluster/test_query_retries.py
index 7989e36..c21f57f 100644
--- a/tests/custom_cluster/test_query_retries.py
+++ b/tests/custom_cluster/test_query_retries.py
@@ -79,7 +79,11 @@ class TestQueryRetries(CustomClusterTestSuite):
 
     # Validate the state of the runtime profiles.
     retried_runtime_profile = self.client.get_runtime_profile(handle)
-    self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
+    retried_query_id =\
+        self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
+
+    # Validate the state of the client log.
+    self.__validate_client_log(handle, retried_query_id)
 
     # Validate the state of the web ui. The query must be closed before validating the
     # state since it asserts that no queries are in flight.
@@ -119,7 +123,11 @@ class TestQueryRetries(CustomClusterTestSuite):
     self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
 
     # Validate the state of the runtime profiles.
-    self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
+    retried_query_id = self.__validate_runtime_profiles(
+        retried_runtime_profile, handle.get_handle().id)
+
+    # Validate the state of the client log.
+    self.__validate_client_log(handle, retried_query_id)
 
     # Validate the state of the web ui. The query must be closed before validating the
     # state since it asserts that no queries are in flight.
@@ -175,7 +183,10 @@ class TestQueryRetries(CustomClusterTestSuite):
       retried_runtime_profile = self.client.get_runtime_profile(handle)
       self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
 
-      self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
+      retried_query_id = self.__validate_runtime_profiles(
+          retried_runtime_profile, handle.get_handle().id)
+
+      self.__validate_client_log(handle, retried_query_id)
 
       self.client.close_query(handle)
 
@@ -217,7 +228,11 @@ class TestQueryRetries(CustomClusterTestSuite):
     self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
 
     # Validate the state of the runtime profiles.
-    self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
+    retried_query_id = self.__validate_runtime_profiles(
+      retried_runtime_profile, handle.get_handle().id)
+
+    # Validate the state of the client log.
+    self.__validate_client_log(handle, retried_query_id)
 
     # Validate the state of the web ui. The query must be closed before validating the
     # state since it asserts that no queries are in flight.
@@ -252,10 +267,12 @@ class TestQueryRetries(CustomClusterTestSuite):
     # Wait until the query fails.
     self.wait_for_state(handle, self.client.QUERY_STATES['EXCEPTION'], 60)
 
-    # The runtime profile of the retried query, need to retrieve the profile
+    # The runtime profile and client log of the retried query, need to be retrieved
     # before fetching results, since the failed fetch attempt will close the
     # query handle.
     retried_runtime_profile = self.client.get_runtime_profile(handle)
+    retried_query_id = self.__get_query_id_from_profile(retried_runtime_profile)
+    self.__validate_client_log(handle, retried_query_id)
 
     # Assert that the query failed, since a query can only be retried once.
     try:
@@ -318,7 +335,9 @@ class TestQueryRetries(CustomClusterTestSuite):
     self.wait_for_state(handle, self.client.QUERY_STATES['FINISHED'], 60)
 
     # Validate that the query was retried.
-    self.__validate_runtime_profiles_from_service(impalad_service, handle)
+    retried_query_id = self.__validate_runtime_profiles_from_service(
+        impalad_service, handle)
+    self.__validate_client_log(handle, retried_query_id)
 
     # Cancel the query.
     self.client.cancel(handle)
@@ -354,7 +373,9 @@ class TestQueryRetries(CustomClusterTestSuite):
     time.sleep(5)
 
     # Validate that the query was retried.
-    self.__validate_runtime_profiles_from_service(impalad_service, handle)
+    retried_query_id = self.__validate_runtime_profiles_from_service(
+        impalad_service, handle)
+    self.__validate_client_log(handle, retried_query_id)
 
     # Assert than attempt to fetch from the query handle fails with a query expired
     # error.
@@ -386,7 +407,8 @@ class TestQueryRetries(CustomClusterTestSuite):
     # Wait for the idle session timeout to expire the session.
     time.sleep(5)
 
-    # Validate that the query was retried.
+    # Validate that the query was retried. Skip validating client log since we can't
+    # get it using the expired session.
     self.__validate_runtime_profiles_from_service(impalad_service, handle)
 
     # Assert than attempt to fetch from the query handle fails with a session expired
@@ -420,8 +442,9 @@ class TestQueryRetries(CustomClusterTestSuite):
     # Validate the state of the runtime profiles.
     retried_runtime_profile = self.hs2_client.get_runtime_profile(handle,
         TRuntimeProfileFormat.STRING)
-    self.__validate_runtime_profiles(retried_runtime_profile,
-        self.hs2_client.get_query_id(handle))
+    retried_query_id = self.__validate_runtime_profiles(
+        retried_runtime_profile, self.hs2_client.get_query_id(handle))
+    self.__validate_client_log(handle, retried_query_id, use_hs2_client=True)
     self.impalad_test_service.wait_for_metric_value(
         'impala-server.resultset-cache.total-num-rows', 1, timeout=60)
     self.hs2_client.close_query(handle)
@@ -433,6 +456,7 @@ class TestQueryRetries(CustomClusterTestSuite):
     retried_query_id = self.__get_retried_query_id_from_profile(original_profile)
     retried_profile = impalad_service.read_query_profile_page(retried_query_id)
     self.__validate_runtime_profiles(retried_profile, handle.get_handle().id)
+    return retried_query_id
 
   def __get_retried_query_id_from_profile(self, profile):
     """Returns the entry for 'Retried Query Id' from the given profile, or 'None' if no
@@ -516,6 +540,7 @@ class TestQueryRetries(CustomClusterTestSuite):
     # Assert that the query options from the original and retried queries are the same.
     assert self.__get_query_options(original_runtime_profile) == \
         self.__get_query_options(retried_runtime_profile)
+    return retried_query_id
 
   def __get_query_options(self, profile):
     """Returns the query options from the given profile."""
@@ -561,3 +586,16 @@ class TestQueryRetries(CustomClusterTestSuite):
     during query execution."""
     assert "Blacklisted Executors: {0}:{1}".format(blacklisted_impalad.hostname,
         blacklisted_impalad.service.be_port) in profile, profile
+
+  def __validate_client_log(self, handle, retried_query_id, use_hs2_client=False):
+    """Validate the GetLog result contains query retry information"""
+    if use_hs2_client:
+      client_log = self.hs2_client.get_log(handle)
+    else:
+      client_log = self.client.get_log(handle)
+    assert "Original query failed:" in client_log
+    query_id_search = re.search("Query has been retried using query id: (.*)\n",
+                                client_log)
+    assert query_id_search,\
+      "Invalid client log, has no retried query id. Log=%s" % client_log
+    assert query_id_search.group(1) == retried_query_id


[impala] 03/03: IMPALA-9890 (Part 1): Add more TPCDS queries to Impala's test suite

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2a48f7dd98c8af64f05f88e17b62c42c076c8bda
Author: Fang-Yu Rao <fa...@cloudera.com>
AuthorDate: Fri Jun 26 16:11:29 2020 -0700

    IMPALA-9890 (Part 1): Add more TPCDS queries to Impala's test suite
    
    This patch adds the following 12 TPCDS queries to the class of
    TestTpcdsDecimalV2Query: Q26, Q30, Q31, Q47, Q48, Q57, Q58, Q59, Q63,
    Q83, Q85, and Q89. All the queries except for Q31 are added to the class
    of TestTpcdsQuery as well because Impala returns one fewer row than
    expected for TestTpcdsQuery::test_tpcds_q31(), which requires further
    investigation.
    
    To verify whether or not the returned result set from Impala for a given
    query is correct, we compare the result set with that produced by the
    HiveServer2 (HS2) in Impala's mini-cluster. We could execute SQL
    statements in HS2 via Beeline, HS2's command line shell, which could be
    launched by the following command.
    
    beeline -u "jdbc:hive2://localhost:11050/default"
    
    We note that among these 12 queries, the execution of Q31, Q58, and Q83
    result in the error of "Counters limit exceeded" by TEZ. To work around
    this problem, for these 3 queries we have to execute the following
    statement before running them to increase the default number of
    counters, which is set to 120.
    
    set tez.counters.max=1200
    
    On the other hand, the table of 'reason' is referenced by Q85. This
    table was not referenced by any TPCDS query before this patch and thus
    was not created. In this regard, in this patch we also modify
    tpcds_schema_template.sql to create this additional table along with its
    data.
    
    Testing:
    - Verified that this patch passes the exhaustive tests in the DEBUG
      build.
    
    Change-Id: Ib5f260e75a3803aabe9ccef271ba94036f96e5cf
    Reviewed-on: http://gerrit.cloudera.org:8080/16119
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 testdata/datasets/tpcds/tpcds_schema_template.sql  |  19 +++
 .../tpcds/queries/tpcds-decimal_v2-q26.test        | 124 +++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q30.test        | 126 +++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q31.test        | 109 +++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q47.test        | 154 +++++++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q48.test        |  71 ++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q57.test        | 151 ++++++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q58.test        |  68 +++++++++
 .../tpcds/queries/tpcds-decimal_v2-q59.test        | 147 ++++++++++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q63.test        |  76 ++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q83.test        |  74 ++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q85.test        |  90 ++++++++++++
 .../tpcds/queries/tpcds-decimal_v2-q89.test        | 131 ++++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q26.test    | 124 +++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q30.test    | 126 +++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q47.test    | 154 +++++++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q48.test    |  71 ++++++++++
 testdata/workloads/tpcds/queries/tpcds-q57.test    | 151 ++++++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q58.test    |  68 +++++++++
 testdata/workloads/tpcds/queries/tpcds-q59.test    | 147 ++++++++++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q63.test    |  76 ++++++++++
 testdata/workloads/tpcds/queries/tpcds-q83.test    |  74 ++++++++++
 testdata/workloads/tpcds/queries/tpcds-q85.test    |  90 ++++++++++++
 testdata/workloads/tpcds/queries/tpcds-q89.test    | 131 ++++++++++++++++++
 tests/query_test/test_tpcds_queries.py             |  69 +++++++++
 tests/util/parse_util.py                           |   2 +-
 26 files changed, 2622 insertions(+), 1 deletion(-)

diff --git a/testdata/datasets/tpcds/tpcds_schema_template.sql b/testdata/datasets/tpcds/tpcds_schema_template.sql
index 91c6c29..079e21a 100644
--- a/testdata/datasets/tpcds/tpcds_schema_template.sql
+++ b/testdata/datasets/tpcds/tpcds_schema_template.sql
@@ -1133,3 +1133,22 @@ SELECT * FROM {db_name}.{table_name};
 LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/web_site/'
 OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
 ====
+---- DATASET
+tpcds
+---- BASE_TABLE_NAME
+reason
+---- COLUMNS
+r_reason_sk           bigint
+r_reason_id           string
+r_reason_desc         string
+---- ROW_FORMAT
+delimited fields terminated by '|'
+---- TABLE_PROPERTIES
+text:serialization.null.format=
+---- DEPENDENT_LOAD
+INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
+SELECT * FROM {db_name}.{table_name};
+---- LOAD
+LOAD DATA LOCAL INPATH '{impala_home}/testdata/impala-data/{db_name}/reason/'
+OVERWRITE INTO TABLE {db_name}{db_suffix}.{table_name};
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q26.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q26.test
new file mode 100644
index 0000000..13c3ae8
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q26.test
@@ -0,0 +1,124 @@
+====
+---- QUERY: TPCDS-Q26
+select  i_item_id,
+        avg(cs_quantity) agg1,
+        avg(cs_list_price) agg2,
+        avg(cs_coupon_amt) agg3,
+        avg(cs_sales_price) agg4
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd_demo_sk and
+       cs_promo_sk = p_promo_sk and
+       cd_gender = 'F' and
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998
+ group by i_item_id
+ order by i_item_id
+ limit 100;
+---- RESULTS
+'AAAAAAAAAAACAAAA',76.0,154.000000,217.690000,9.240000
+'AAAAAAAAAABAAAAA',8.0,52.110000,0.000000,44.290000
+'AAAAAAAAAABBAAAA',53.0,35.020000,0.000000,3.500000
+'AAAAAAAAAACDAAAA',28.0,153.650000,0.000000,136.740000
+'AAAAAAAAAADBAAAA',9.5,43.050000,0.000000,10.190000
+'AAAAAAAAAADEAAAA',70.0,35.220000,0.000000,26.410000
+'AAAAAAAAAAEAAAAA',26.0,119.485000,0.000000,24.735000
+'AAAAAAAAAAFCAAAA',20.0,199.230000,0.000000,27.890000
+'AAAAAAAAAAHAAAAA',17.0,39.420000,0.000000,12.220000
+'AAAAAAAAAAHBAAAA',72.0,81.190000,0.000000,68.190000
+'AAAAAAAAAAIAAAAA',44.0,76.980000,0.000000,7.690000
+'AAAAAAAAAAICAAAA',70.0,189.360000,0.000000,176.100000
+'AAAAAAAAAAIDAAAA',12.0,68.970000,0.000000,66.210000
+'AAAAAAAAAAKBAAAA',24.0,146.990000,0.000000,70.550000
+'AAAAAAAAAAKDAAAA',98.0,35.030000,0.000000,14.360000
+'AAAAAAAAAALAAAAA',45.0,152.860000,1232.640000,85.600000
+'AAAAAAAAAALDAAAA',8.0,148.550000,115.270000,144.090000
+'AAAAAAAAAAOAAAAA',16.0,58.170000,0.000000,48.280000
+'AAAAAAAAAAODAAAA',81.0,97.915000,746.790000,69.140000
+'AAAAAAAAAAPBAAAA',42.0,104.440000,0.000000,97.120000
+'AAAAAAAAAAPCAAAA',11.0,77.940000,0.000000,18.700000
+'AAAAAAAAABAAAAAA',55.0,121.156667,320.546667,76.123333
+'AAAAAAAAABADAAAA',42.5,180.665000,0.000000,131.275000
+'AAAAAAAAABAEAAAA',49.0,169.480000,0.000000,27.590000
+'AAAAAAAAABBAAAAA',51.0,29.470000,0.000000,13.550000
+'AAAAAAAAABBDAAAA',14.0,15.100000,34.120000,12.830000
+'AAAAAAAAABCBAAAA',74.0,191.510000,0.000000,61.280000
+'AAAAAAAAABDAAAAA',80.0,69.265000,959.435000,29.440000
+'AAAAAAAAABDBAAAA',97.0,65.130000,2430.470000,33.860000
+'AAAAAAAAABFBAAAA',16.0,86.310000,0.000000,63.860000
+'AAAAAAAAABFCAAAA',31.0,1.370000,0.000000,1.130000
+'AAAAAAAAABGDAAAA',47.0,165.520000,293.280000,48.000000
+'AAAAAAAAABHDAAAA',61.0,177.800000,690.750000,23.110000
+'AAAAAAAAABIBAAAA',72.0,139.090000,0.000000,27.810000
+'AAAAAAAAABJAAAAA',45.0,36.290000,0.000000,0.000000
+'AAAAAAAAABKAAAAA',77.0,107.030000,0.000000,48.160000
+'AAAAAAAAABKDAAAA',100.0,16.450000,0.000000,4.600000
+'AAAAAAAAABLCAAAA',1.0,3.430000,0.000000,1.740000
+'AAAAAAAAABNAAAAA',54.5,68.895000,0.000000,21.380000
+'AAAAAAAAABNCAAAA',43.5,120.585000,142.815000,73.435000
+'AAAAAAAAABOBAAAA',13.0,63.930000,0.000000,0.000000
+'AAAAAAAAABPAAAAA',28.0,53.250000,0.000000,13.310000
+'AAAAAAAAABPDAAAA',40.25,142.672500,58.732500,47.195000
+'AAAAAAAAACBBAAAA',94.0,69.530000,0.000000,52.140000
+'AAAAAAAAACBCAAAA',94.0,63.220000,0.000000,34.130000
+'AAAAAAAAACCAAAAA',66.0,17.100000,0.000000,16.920000
+'AAAAAAAAACCBAAAA',62.0,106.890000,0.000000,16.030000
+'AAAAAAAAACCDAAAA',34.0,103.070000,0.000000,51.530000
+'AAAAAAAAACDAAAAA',46.5,170.480000,733.160000,60.520000
+'AAAAAAAAACDCAAAA',28.0,162.280000,0.000000,108.720000
+'AAAAAAAAACFBAAAA',4.0,2.010000,0.000000,0.940000
+'AAAAAAAAACFEAAAA',23.0,67.390000,0.000000,43.120000
+'AAAAAAAAACGAAAAA',25.0,111.470000,0.000000,84.710000
+'AAAAAAAAACGCAAAA',34.0,131.030000,0.000000,74.860000
+'AAAAAAAAACGDAAAA',44.5,57.510000,0.000000,11.755000
+'AAAAAAAAACIAAAAA',68.3333333333,112.600000,0.000000,57.823333
+'AAAAAAAAACKBAAAA',78.0,84.745000,679.460000,26.350000
+'AAAAAAAAACKCAAAA',51.5,133.955000,0.000000,109.205000
+'AAAAAAAAACLAAAAA',58.5,127.340000,9.225000,11.575000
+'AAAAAAAAACLBAAAA',86.0,39.730000,0.000000,3.970000
+'AAAAAAAAACMAAAAA',68.0,10.300000,0.000000,2.470000
+'AAAAAAAAACOBAAAA',42.5,14.955000,0.000000,14.000000
+'AAAAAAAAACODAAAA',68.0,40.410000,0.000000,9.290000
+'AAAAAAAAADBAAAAA',55.5,68.335000,25.585000,63.210000
+'AAAAAAAAADBDAAAA',69.0,25.330000,0.000000,9.370000
+'AAAAAAAAADCCAAAA',48.0,100.580000,0.000000,28.160000
+'AAAAAAAAADEAAAAA',44.5,94.815000,955.395000,51.705000
+'AAAAAAAAADEBAAAA',48.0,47.450000,0.000000,3.320000
+'AAAAAAAAADEDAAAA',71.0,150.090000,0.000000,112.560000
+'AAAAAAAAADEEAAAA',36.0,66.390000,0.000000,17.920000
+'AAAAAAAAADFAAAAA',21.5,115.710000,1641.000000,102.130000
+'AAAAAAAAADFCAAAA',32.5,113.635000,0.000000,67.835000
+'AAAAAAAAADFDAAAA',32.0,47.500000,0.000000,23.750000
+'AAAAAAAAADHAAAAA',79.0,228.170000,0.000000,73.010000
+'AAAAAAAAADICAAAA',74.0,131.620000,0.000000,101.340000
+'AAAAAAAAADKDAAAA',1.0,127.880000,83.240000,89.510000
+'AAAAAAAAADLCAAAA',84.0,27.840000,247.970000,18.620000
+'AAAAAAAAADNAAAAA',56.0,207.020000,0.000000,163.540000
+'AAAAAAAAADNBAAAA',81.0,21.230000,0.000000,0.840000
+'AAAAAAAAADOAAAAA',32.0,15.375000,0.000000,6.330000
+'AAAAAAAAADPBAAAA',5.0,49.320000,0.000000,8.870000
+'AAAAAAAAAEADAAAA',68.0,26.810000,0.000000,26.000000
+'AAAAAAAAAEBAAAAA',34.0,206.220000,0.000000,181.470000
+'AAAAAAAAAECEAAAA',42.0,177.930000,0.000000,86.520000
+'AAAAAAAAAEDAAAAA',5.0,66.020000,304.040000,62.050000
+'AAAAAAAAAEDDAAAA',73.0,192.700000,0.000000,21.190000
+'AAAAAAAAAEECAAAA',84.0,93.980000,0.000000,93.040000
+'AAAAAAAAAEGBAAAA',25.0,221.440000,0.000000,24.350000
+'AAAAAAAAAEHAAAAA',49.0,3.280000,0.000000,3.240000
+'AAAAAAAAAEHCAAAA',10.0,66.100000,0.000000,46.930000
+'AAAAAAAAAEHDAAAA',68.0,88.660000,0.000000,88.660000
+'AAAAAAAAAEJBAAAA',48.0,228.350000,0.000000,150.710000
+'AAAAAAAAAEJDAAAA',51.0,102.020000,0.000000,4.080000
+'AAAAAAAAAEKAAAAA',88.0,50.120000,527.150000,11.520000
+'AAAAAAAAAEKCAAAA',51.75,123.510000,742.540000,85.630000
+'AAAAAAAAAELBAAAA',95.0,8.370000,0.000000,7.700000
+'AAAAAAAAAEMBAAAA',13.0,120.050000,340.800000,33.610000
+'AAAAAAAAAENCAAAA',90.0,81.430000,0.000000,77.350000
+'AAAAAAAAAEOBAAAA',86.0,92.170000,0.000000,0.000000
+'AAAAAAAAAEOCAAAA',76.0,40.170000,0.000000,28.520000
+---- TYPES
+STRING, DOUBLE, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q30.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q30.test
new file mode 100644
index 0000000..0d5f5eb
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q30.test
@@ -0,0 +1,126 @@
+====
+---- QUERY: TPCDS-Q30
+with customer_total_return as
+ (select wr_returning_customer_sk as ctr_customer_sk
+        ,ca_state as ctr_state,
+    sum(wr_return_amt) as ctr_total_return
+ from web_returns
+     ,date_dim
+     ,customer_address
+ where wr_returned_date_sk = d_date_sk
+   and d_year =2002
+   and wr_returning_addr_sk = ca_address_sk
+ group by wr_returning_customer_sk
+         ,ca_state)
+  select  c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+       ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+       ,c_last_review_date,ctr_total_return
+ from customer_total_return ctr1
+     ,customer_address
+     ,customer
+ where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+              from customer_total_return ctr2
+                          where ctr1.ctr_state = ctr2.ctr_state)
+       and ca_address_sk = c_current_addr_sk
+       and ca_state = 'IL'
+       and ctr1.ctr_customer_sk = c_customer_sk
+ order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+                  ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+                  ,c_last_review_date,ctr_total_return
+limit 100;
+---- RESULTS
+'AAAAAAAAACBFAAAA','Dr.','Clifton','Carr','N',4,4,1950,'INDIA','NULL','Clifton.Carr@mOm0RG.com','2452385',2232.42
+'AAAAAAAAADCCAAAA','Sir','Elmer','Southerland','Y',14,3,1967,'NICARAGUA','NULL','Elmer.Southerland@dG9HZHGjU8.edu','2452623',9604.53
+'AAAAAAAAADFDBAAA','Dr.','Stanley','Smith','N',23,4,1928,'SPAIN','NULL','Stanley.Smith@Yxk9F0obDmkvN.org','2452381',2287.41
+'AAAAAAAAAEHLAAAA','Ms.','Anita','Vance','Y',14,3,1988,'ZIMBABWE','NULL','Anita.Vance@SCZrSvm7MpKZ.org','2452362',4263.60
+'AAAAAAAAAIJOAAAA','Dr.','Elise','Cook','Y',25,1,1957,'SAN MARINO','NULL','Elise.Cook@zgV4U.com','2452537',1940.00
+'AAAAAAAAAJLEBAAA','Mrs.','Judy','Wells','NULL',31,8,NULL,'JERSEY','NULL','NULL','NULL',1764.00
+'AAAAAAAAAMHMAAAA','Mrs.','Angel','Farrington','N',5,5,1992,'NICARAGUA','NULL','Angel.Farrington@ESvsPDBzslNpvaZ.org','2452593',3722.86
+'AAAAAAAABEGGAAAA','Ms.','Linda','Bishop','N',25,5,1945,'ANDORRA','NULL','Linda.Bishop@SsgvULplErri.org','2452369',5049.28
+'AAAAAAAABLPMAAAA','Sir','Peter','Hogue','N',24,4,1979,'FRENCH GUIANA','NULL','Peter.Hogue@XZhvIIKsznzvj.com','2452640',1582.00
+'AAAAAAAABNBKAAAA','Dr.','Chelsea','Wellman','Y',5,2,1989,'NIGERIA','NULL','Chelsea.Wellman@RY0lIpgj89SBPI.com','2452454',4034.24
+'AAAAAAAABNECAAAA','Dr.','Thomas','Hollingsworth','N',30,9,1960,'AFGHANISTAN','NULL','Thomas.Hollingsworth@GM6CNHy.org','2452581',6428.07
+'AAAAAAAABNMAAAAA','Miss','Martha','Moore','N',5,7,1985,'BOUVET ISLAND','NULL','Martha.Moore@s.org','2452584',7708.40
+'AAAAAAAABNMEBAAA','Dr.','Sara','Mcmahon','N',23,10,1926,'BARBADOS','NULL','Sara.Mcmahon@DGmEehYquCX.edu','2452584',1290.24
+'AAAAAAAACDKGAAAA','Dr.','Martha','Mason','N',20,6,1970,'MOROCCO','NULL','Martha.Mason@F.org','2452359',3861.33
+'AAAAAAAACEONAAAA','Ms.','Sandra','Lopez','Y',31,3,1979,'EGYPT','NULL','Sandra.Lopez@pJc3Or8NbSu8MX.com','2452311',4688.40
+'AAAAAAAACGBPAAAA','Ms.','Florence','Cabrera','Y',30,6,1934,'GHANA','NULL','Florence.Cabrera@cm.com','2452610',1608.64
+'AAAAAAAACIGIBAAA','Dr.','Charles','Lopez','Y',11,10,1979,'URUGUAY','NULL','Charles.Lopez@gOyo.com','2452424',2025.00
+'AAAAAAAACIOGBAAA','Dr.','Gregorio','Lee','N',3,10,1970,'LESOTHO','NULL','Gregorio.Lee@iteZa836XrfF.edu','2452438',1590.30
+'AAAAAAAACJLIAAAA','Mr.','Philip','Kinder','N',1,10,1987,'BOLIVIA','NULL','Philip.Kinder@t4B5veuzCR390MP.com','2452595',2526.44
+'AAAAAAAACKDBAAAA','Miss','Kathy','Kelly','N',15,9,1988,'ANTIGUA AND BARBUDA','NULL','Kathy.Kelly@o.org','2452339',2719.92
+'AAAAAAAACPLIAAAA','Mr.','John','Williams','Y',11,9,1960,'COMOROS','NULL','John.Williams@Gn.com','2452392',4542.08
+'AAAAAAAACPOIAAAA','Dr.','Melvin','Phillips','Y',16,2,1987,'LESOTHO','NULL','Melvin.Phillips@MinR9.com','2452402',2647.26
+'AAAAAAAADADNAAAA','Dr.','Willie','Vitale','Y',30,11,1978,'SLOVAKIA','NULL','Willie.Vitale@rUnNJoaskyn6aLA.edu','2452642',6185.28
+'AAAAAAAADAGBAAAA','Ms.','Helen','Barker','N',10,5,1933,'ZAMBIA','NULL','Helen.Barker@V8ZT4ncYBuGSK.com','2452378',2791.32
+'AAAAAAAADNLLAAAA','Mrs.','Mae','Jackson','N',15,8,1948,'SYRIAN ARAB REPUBLIC','NULL','Mae.Jackson@Eh2TmyLnCy.org','2452379',5097.08
+'AAAAAAAADPCIAAAA','Miss','Wilma','Sanborn','N',15,6,1952,'BURUNDI','NULL','Wilma.Sanborn@2jbbN0ubazbL.org','2452333',2457.60
+'AAAAAAAAEHFFBAAA','Dr.','Michelle','Reed','N',1,3,1982,'ZAMBIA','NULL','Michelle.Reed@9yNA4PLMEjB.edu','2452546',5864.34
+'AAAAAAAAFABKAAAA','Dr.','Nancy','Duke','N',10,1,1961,'ERITREA','NULL','Nancy.Duke@h.edu','2452379',1607.04
+'AAAAAAAAFCKHBAAA','Mrs.','Helen','Ramsay','N',6,1,1971,'LIECHTENSTEIN','NULL','Helen.Ramsay@ZC1VJrVtx7It3Iz.com','2452626',2868.48
+'AAAAAAAAFGBABAAA','Mrs.','Louise','Douglas','N',12,10,1944,'ARUBA','NULL','Louise.Douglas@VzomaEVuSe.edu','2452630',9067.96
+'AAAAAAAAFPAIBAAA','Dr.','Michael','Roberge','N',29,1,1959,'VIET NAM','NULL','Michael.Roberge@YTN9dUXCpJmz5.org','2452378',4120.84
+'AAAAAAAAGAMHAAAA','Miss','Clementine','Darby','N',5,2,1952,'SAN MARINO','NULL','Clementine.Darby@8HfgoxcVueO.com','2452573',2829.50
+'AAAAAAAAGBJKAAAA','Dr.','Amy','Baker','Y',5,6,1970,'SAUDI ARABIA','NULL','Amy.Baker@U8marrVO5XN9oMK.com','2452370',2981.21
+'AAAAAAAAGDFJAAAA','Ms.','Margaret','Edmonds','NULL',24,NULL,NULL,'ARUBA','NULL','NULL','NULL',3097.44
+'AAAAAAAAGHKPAAAA','Miss','Kimberly','Clark','N',15,11,1937,'TONGA','NULL','Kimberly.Clark@xYl6FQTOMkQPi.edu','2452477',1852.52
+'AAAAAAAAGKPGAAAA','Mrs.','Rachel','Reed','Y',13,3,1950,'MALAWI','NULL','Rachel.Reed@x.com','2452574',2254.35
+'AAAAAAAAGNKPAAAA','Dr.','Marcus','Davis','N',12,2,1968,'KENYA','NULL','Marcus.Davis@ksKq.com','2452633',2388.68
+'AAAAAAAAGNKPAAAA','Dr.','Marcus','Davis','N',12,2,1968,'KENYA','NULL','Marcus.Davis@ksKq.com','2452633',5162.50
+'AAAAAAAAGPBFAAAA','Dr.','Lorenzo','Grace','Y',5,2,1984,'UNITED STATES','NULL','Lorenzo.Grace@M8CTliq9t0H6.edu','2452613',2904.66
+'AAAAAAAAHDBLAAAA','Mr.','Michael','Dupre','Y',5,9,1961,'TIMOR-LESTE','NULL','Michael.Dupre@AhCfsll84OUMVNZ.com','2452438',3593.72
+'AAAAAAAAHDKHBAAA','Sir','NULL','Free','N',6,NULL,1956,'NULL','NULL','NULL','2452293',2100.64
+'AAAAAAAAHEFEAAAA','Dr.','James','Martin','Y',28,9,1963,'PERU','NULL','James.Martin@31H8tf1eym.org','2452321',2928.24
+'AAAAAAAAHKCEBAAA','Mrs.','Stella','Osorio','N',2,10,1966,'ANGUILLA','NULL','Stella.Osorio@6M8UcU9r.org','2452421',2095.08
+'AAAAAAAAHNIFBAAA','Mrs.','Jodi','Hickman','N',15,9,1973,'MALAYSIA','NULL','Jodi.Hickman@OdAHHq4.com','2452312',8251.12
+'AAAAAAAAIFNHBAAA','Sir','Victor','Mccloskey','N',24,2,1940,'MONTSERRAT','NULL','Victor.Mccloskey@RymmApu9aLZ.com','2452604',1819.48
+'AAAAAAAAIGJGAAAA','Sir','Edward','Britton','Y',NULL,NULL,1981,'NULL','NULL','Edward.Britton@G.edu','NULL',3791.04
+'AAAAAAAAIHLGAAAA','Miss','Jennifer','Smith','N',31,10,1969,'TRINIDAD AND TOBAGO','NULL','Jennifer.Smith@vYc7AUOo7.org','2452564',7360.74
+'AAAAAAAAIIABAAAA','Miss','Rosa','Phillips','Y',19,3,1949,'SURINAME','NULL','Rosa.Phillips@ECXsk.com','2452488',6261.20
+'AAAAAAAAIIHMAAAA','Mr.','Ricky','Gerber','N',10,11,1973,'WALLIS AND FUTUNA','NULL','Ricky.Gerber@5dsb4vhLdmZm.edu','2452402',1707.12
+'AAAAAAAAIKABAAAA','Dr.','James','Taylor','Y',2,1,1967,'LEBANON','NULL','James.Taylor@eVoZuvYcqACN.org','2452537',5759.52
+'AAAAAAAAIMNPAAAA','Dr.','John','Mendoza','Y',7,1,1940,'HAITI','NULL','John.Mendoza@XDX7Ku8gVsR1LHn.org','2452462',3580.20
+'AAAAAAAAIMPAAAAA','Ms.','Lois','Washington','N',21,8,1973,'TONGA','NULL','Lois.Washington@uvUAMB9DuxcK.edu','2452319',1756.80
+'AAAAAAAAJAMHBAAA','Miss','Kathleen','Dupre','Y',7,10,1951,'TURKEY','NULL','Kathleen.Dupre@hfQ.edu','2452349',6127.60
+'AAAAAAAAJEIFBAAA','NULL','NULL','NULL','N',NULL,NULL,1967,'NULL','NULL','Michael.Lee@k0QIl6mntspX.org','2452411',3304.87
+'AAAAAAAAJGJEBAAA','Mr.','Charles','Jackson','N',25,3,1945,'LIBERIA','NULL','Charles.Jackson@ziLxUxp0.edu','2452585',2983.56
+'AAAAAAAAJHBDBAAA','Mr.','Edgar','Hunt','Y',27,12,1932,'CAMEROON','NULL','Edgar.Hunt@Z.edu','2452284',2066.48
+'AAAAAAAAJJKMAAAA','Sir','Jimmy','Mcinnis','Y',25,1,1964,'SINGAPORE','NULL','Jimmy.Mcinnis@9QXGY2Fn18bZ.org','2452483',3542.97
+'AAAAAAAAJKACAAAA','Dr.','Arthur','Harris','N',9,3,1925,'SEYCHELLES','NULL','Arthur.Harris@Sj3QkYzfu.com','2452622',4920.90
+'AAAAAAAAKBOHAAAA','Mr.','NULL','Christian','NULL',NULL,10,NULL,'GUERNSEY','NULL','James.Christian@8r5T.com','2452522',2824.24
+'AAAAAAAAKGFEBAAA','Mr.','Federico','Bradshaw','N',11,5,1978,'MONTSERRAT','NULL','Federico.Bradshaw@uzthL55V.com','2452453',1633.59
+'AAAAAAAAKJDMAAAA','Miss','Andrew','Osborne','N',13,5,1954,'SUDAN','NULL','Andrew.Osborne@D.edu','2452495',2858.60
+'AAAAAAAAKKCMAAAA','Sir','Randall','Oliver','Y',15,6,1969,'GREECE','NULL','Randall.Oliver@lr7eEGq2qUT8sD2.edu','2452305',3917.80
+'AAAAAAAAKMJBAAAA','Sir','John','Kruse','N',25,9,1950,'NEW ZEALAND','NULL','John.Kruse@BgZ6x.edu','2452516',4032.72
+'AAAAAAAAKPPEBAAA','Mr.','Thomas','Kurtz','N',23,1,1953,'WALLIS AND FUTUNA','NULL','Thomas.Kurtz@Xd5JPu7M1.edu','2452571',2123.82
+'AAAAAAAALCECAAAA','Sir','Dennis','Silva','N',29,10,1972,'TAJIKISTAN','NULL','Dennis.Silva@cjk.com','2452425',1934.64
+'AAAAAAAALFAGBAAA','Sir','Alvin','Young','Y',24,10,1973,'MONACO','NULL','Alvin.Young@iit.org','2452518',4766.65
+'AAAAAAAALJMHBAAA','Dr.','Stephanie','Maloney','Y',19,12,1935,'MOZAMBIQUE','NULL','Stephanie.Maloney@xlkHMrqUSFcrHG.com','2452572',14950.15
+'AAAAAAAALKHEAAAA','Mr.','Neil','Delarosa','Y',6,1,1933,'SEYCHELLES','NULL','Neil.Delarosa@F26REbIyVtpkY.org','2452299',1575.84
+'AAAAAAAALPIOAAAA','Mr.','Kendrick','Swank','N',10,9,1983,'NIGERIA','NULL','Kendrick.Swank@KiHF1kfJUPMgrkxAtGtI.com','2452629',6283.76
+'AAAAAAAAMAHMAAAA','Sir','John','Nunez','Y',29,7,1963,'GIBRALTAR','NULL','John.Nunez@NXXNFCCxOFqDq6B.com','2452400',8665.51
+'AAAAAAAAMBIEAAAA','Dr.','Timothy','Bruce','Y',25,3,1938,'FIJI','NULL','Timothy.Bruce@gO.org','2452530',3006.00
+'AAAAAAAAMFGLAAAA','Dr.','Ellis','Stringer','N',30,3,1985,'VIET NAM','NULL','Ellis.Stringer@dLx2T0e7gAp.com','2452520',1646.55
+'AAAAAAAAMFNCAAAA','Miss','Sandra','Chavez','N',15,10,1936,'MAURITANIA','NULL','Sandra.Chavez@jNLsx1jyV7C.org','2452610',7062.75
+'AAAAAAAAMKHOAAAA','Miss','Kristie','Ross','N',6,1,1966,'BARBADOS','NULL','Kristie.Ross@p.edu','2452439',1842.95
+'AAAAAAAANDGCBAAA','Ms.','Sheila','Hull','N',23,11,1992,'EQUATORIAL GUINEA','NULL','Sheila.Hull@eDEnMk74Ukoo.com','2452416',3453.32
+'AAAAAAAANIAFBAAA','Dr.','Adrian','Holt','Y',27,3,1990,'SEYCHELLES','NULL','Adrian.Holt@eQCL.org','2452529',1233.10
+'AAAAAAAANJJGAAAA','Dr.','David','Smith','Y',20,7,1939,'GUINEA-BISSAU','NULL','David.Smith@q8KF56o7peqsXNf3I.com','2452371',3186.92
+'AAAAAAAANNBABAAA','Ms.','Cordelia','Carpenter','Y',19,8,1990,'PALAU','NULL','Cordelia.Carpenter@nsyrjxnu.com','2452487',5297.04
+'AAAAAAAANONGAAAA','Dr.','Joel','Rucker','N',15,2,1947,'BOLIVIA','NULL','Joel.Rucker@CaD.edu','2452289',1910.50
+'AAAAAAAAOCGFAAAA','Miss','Adam','Martin','Y',20,9,1970,'ARMENIA','NULL','Adam.Martin@0Sb2XqFA9.edu','2452626',3373.66
+'AAAAAAAAOFGEAAAA','Sir','John','Hopkins','N',9,4,1953,'MALTA','NULL','John.Hopkins@zsuCL.edu','2452508',5102.37
+'AAAAAAAAOFICAAAA','NULL','Ramona','Moore','N',NULL,NULL,1949,'NULL','NULL','Ramona.Moore@Zk.edu','NULL',1641.74
+'AAAAAAAAOFIHBAAA','Mr.','Francis','Wilhelm','Y',28,1,1985,'LEBANON','NULL','Francis.Wilhelm@R3.edu','2452291',11642.40
+'AAAAAAAAOLBHBAAA','Dr.','Nathaniel','Downs','Y',2,7,1944,'DOMINICA','NULL','Nathaniel.Downs@sdfiKzLFg7Et.org','2452514',7484.11
+'AAAAAAAAOLBHBAAA','Dr.','Nathaniel','Downs','Y',2,7,1944,'DOMINICA','NULL','Nathaniel.Downs@sdfiKzLFg7Et.org','2452514',8594.42
+'AAAAAAAAOMMCAAAA','Dr.','Roberta','Johnson','Y',15,3,1976,'ANTIGUA AND BARBUDA','NULL','Roberta.Johnson@uAvA75ZjM.com','2452581',1803.33
+'AAAAAAAAPEAIAAAA','Sir','William','Mason','Y',1,5,1977,'KYRGYZSTAN','NULL','William.Mason@RD4e6I1JfQkEL9.org','2452495',3683.03
+'AAAAAAAAPIHKAAAA','Dr.','Felix','Bailey','Y',8,7,1959,'TIMOR-LESTE','NULL','Felix.Bailey@0dKxiSYTnCjo5ZPe1.com','2452328',2856.00
+'AAAAAAAAPINOAAAA','Dr.','Michael','Johnson','Y',3,1,1976,'URUGUAY','NULL','Michael.Johnson@5l6aamKSUv6BV.org','2452531',8948.80
+'AAAAAAAAPJAGBAAA','Dr.','Michael','Stevens','N',28,2,1943,'GEORGIA','NULL','Michael.Stevens@c9ygsUdHa3J1XG.com','2452349',4491.28
+'AAAAAAAAPKCMAAAA','Dr.','Harold','Williams','N',13,9,1976,'QATAR','NULL','Harold.Williams@U44mtEQN0Qrm.org','2452409',1760.85
+'AAAAAAAAPNDDAAAA','Mr.','Tomas','Menendez','N',26,9,1934,'FINLAND','NULL','Tomas.Menendez@qibyGN3Ior.edu','2452573',1945.48
+---- TYPES
+STRING, STRING, STRING, STRING, STRING, INT, INT, INT, STRING, STRING, STRING, STRING, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q31.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q31.test
new file mode 100644
index 0000000..5a690cf
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q31.test
@@ -0,0 +1,109 @@
+====
+---- QUERY: TPCDS-Q31
+-- THE RESULT ROWS ARE SORTED ACCORDING TO THE COLUMN OF SS1.D_YEAR. BUT SINCE THE COLUMN
+-- OF SS1.D_YEAR IS 2000 FOR EACH RETURNED ROW, THE RESULT SET SHOULD BE CONSIDERED
+-- CORRECT REGARDLESS OF THE RELATIVE ORDERING OF RETURNED ROWS.
+with ss as
+ (select ca_county,d_qoy, d_year,sum(ss_ext_sales_price) as store_sales
+ from store_sales,date_dim,customer_address
+ where ss_sold_date_sk = d_date_sk
+  and ss_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year),
+ ws as
+ (select ca_county,d_qoy, d_year,sum(ws_ext_sales_price) as web_sales
+ from web_sales,date_dim,customer_address
+ where ws_sold_date_sk = d_date_sk
+  and ws_bill_addr_sk=ca_address_sk
+ group by ca_county,d_qoy, d_year)
+ select /* tt */
+        ss1.ca_county
+       ,ss1.d_year
+       ,ws2.web_sales/ws1.web_sales web_q1_q2_increase
+       ,ss2.store_sales/ss1.store_sales store_q1_q2_increase
+       ,ws3.web_sales/ws2.web_sales web_q2_q3_increase
+       ,ss3.store_sales/ss2.store_sales store_q2_q3_increase
+ from
+        ss ss1
+       ,ss ss2
+       ,ss ss3
+       ,ws ws1
+       ,ws ws2
+       ,ws ws3
+ where
+    ss1.d_qoy = 1
+    and ss1.d_year = 2000
+    and ss1.ca_county = ss2.ca_county
+    and ss2.d_qoy = 2
+    and ss2.d_year = 2000
+ and ss2.ca_county = ss3.ca_county
+    and ss3.d_qoy = 3
+    and ss3.d_year = 2000
+    and ss1.ca_county = ws1.ca_county
+    and ws1.d_qoy = 1
+    and ws1.d_year = 2000
+    and ws1.ca_county = ws2.ca_county
+    and ws2.d_qoy = 2
+    and ws2.d_year = 2000
+    and ws1.ca_county = ws3.ca_county
+    and ws3.d_qoy = 3
+    and ws3.d_year =2000
+    and case when ws1.web_sales > 0 then ws2.web_sales/ws1.web_sales else null end
+       > case when ss1.store_sales > 0 then ss2.store_sales/ss1.store_sales else null end
+    and case when ws2.web_sales > 0 then ws3.web_sales/ws2.web_sales else null end
+       > case when ss2.store_sales > 0 then ss3.store_sales/ss2.store_sales else null end
+ order by ss1.d_year;
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'Oxford County',2000,0.973143,0.757240,4.012687,1.640751
+'Atchison County',2000,0.800221,0.242964,11.944560,3.162954
+'Bradley County',2000,1.489632,0.575785,1.344390,0.998919
+'Mitchell County',2000,4.439233,1.161335,1.394036,1.256074
+'Sheridan County',2000,1.386030,1.250657,1.575936,0.537886
+'Smith County',2000,0.636936,0.427882,5.744845,4.477958
+'Refugio County',2000,1.812976,0.586732,1.301984,1.269604
+'Williamson County',2000,2.985102,0.391418,5.805965,4.396699
+'Nicholas County',2000,2.165116,2.056274,6.021300,1.262577
+'Ingham County',2000,0.574390,0.384114,1.306517,0.992061
+'Marion County',2000,1.158999,0.916511,2.445040,1.851073
+'Bacon County',2000,1.168826,0.394038,0.968800,0.510743
+'Edmonson County',2000,0.732442,0.299753,1.602897,1.491246
+'Vernon County',2000,0.974454,0.915946,1.368804,1.041712
+'Lamar County',2000,0.749358,0.745617,4.269036,2.029584
+'Harlan County',2000,1.670355,1.590115,2.471959,2.107294
+'Forest County',2000,0.644745,0.342367,5.771027,1.881074
+'Stark County',2000,7.338220,1.417589,1.863840,1.227366
+'Gaston County',2000,0.763774,0.455496,3.955611,2.141572
+'Boyd County',2000,1.086324,0.811507,1.168972,0.742057
+'Heard County',2000,4.102554,1.266947,3.500228,1.127804
+'Miller County',2000,2.575745,1.318273,2.191968,0.982218
+'Richmond County',2000,1.571665,1.294015,2.309592,1.778016
+'Mora County',2000,1.183261,0.635654,2.513120,0.918567
+'Nantucket County',2000,1.437897,0.722692,1.175488,0.962304
+'Lincoln County',2000,1.019137,0.944845,2.335960,1.776928
+'Stone County',2000,1.900042,0.811920,3.699361,1.521662
+'Meriwether County',2000,0.365748,0.300417,2.772270,0.787769
+'Ferry County',2000,0.701373,0.341010,4.002919,2.603464
+'Perry County',2000,1.580780,0.764453,2.153370,1.802410
+'Wright County',2000,5.029335,1.970810,4.076529,1.966472
+'Prince William County',2000,3.376373,0.630787,1.706967,0.934324
+'Cass County',2000,2.398720,1.190582,2.257169,0.847801
+'Fillmore County',2000,0.507769,0.349921,2.443162,1.301136
+'Green County',2000,0.763667,0.321499,4.694152,4.207034
+'Houston County',2000,2.045326,1.038983,1.965051,1.421193
+'Grant County',2000,0.693336,0.622870,1.788644,1.722197
+'Steele County',2000,1.377413,0.766513,1.247973,0.931195
+'Mercer County',2000,0.738342,0.601678,3.012450,2.724470
+'Culpeper County',2000,0.661751,0.619014,1.659248,1.224169
+'Harris County',2000,2.337569,0.333126,2.417058,1.025794
+'Corson County',2000,0.560309,0.175081,4.807409,3.227141
+'Otero County',2000,2.754655,1.246299,2.976111,2.245835
+'Rice County',2000,1.134698,0.733018,2.378149,1.986402
+'Crockett County',2000,1.637106,0.360468,2.134042,1.832451
+'Lake County',2000,1.255077,0.745928,1.515147,1.265662
+'New Kent County',2000,0.602616,0.399062,2.869386,2.625895
+'Bourbon County',2000,1.913115,0.981928,3.364802,1.380123
+'Tooele County',2000,6.590303,0.768910,1.788637,0.340067
+'Carter County',2000,3.953739,1.151033,2.119285,1.844491
+'Buchanan County',2000,1.195667,0.746049,3.332904,2.239788
+---- TYPES
+STRING, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q47.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q47.test
new file mode 100644
index 0000000..08ef70d
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q47.test
@@ -0,0 +1,154 @@
+====
+---- QUERY: TPCDS-Q47
+with v1 as(
+ select i_category, i_brand,
+        s_store_name, s_company_name,
+        d_year, d_moy,
+        sum(ss_sales_price) sum_sales,
+        avg(sum(ss_sales_price)) over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name
+           order by d_year, d_moy) rn
+ from item, store_sales, date_dim, store
+ where ss_item_sk = i_item_sk and
+       ss_sold_date_sk = d_date_sk and
+       ss_store_sk = s_store_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          s_store_name, s_company_name,
+          d_year, d_moy),
+ v2 as(
+ select v1.i_category
+        ,v1.d_year, v1.d_moy
+        ,v1.avg_monthly_sales
+        ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1.s_store_name = v1_lag.s_store_name and
+       v1.s_store_name = v1_lead.s_store_name and
+       v1.s_company_name = v1_lag.s_company_name and
+       v1.s_company_name = v1_lead.s_company_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+  select  *
+ from v2
+ where  d_year = 2000 and
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, 3
+ limit 100;
+---- RESULTS
+'Women',2000,6,5167.868333,1702.17,3236.87,3399.13
+'Music',2000,4,5036.090833,1860.99,2706.82,3303.43
+'Children',2000,3,5064.604167,1912.86,3298.07,2757.48
+'Women',2000,3,4913.032500,1779.75,2390.63,2257.04
+'Shoes',2000,7,5168.767500,2069.14,2983.37,8035.14
+'Music',2000,3,5110.744167,2022.81,2461.84,3067.59
+'Music',2000,5,4803.555833,1743.74,2778.83,3066.97
+'Children',2000,7,5175.091667,2116.90,3610.19,6870.27
+'Men',2000,4,4937.400833,1880.07,3078.36,2756.00
+'Women',2000,3,4958.763333,1958.78,2678.23,3027.08
+'Music',2000,3,5212.675833,2215.63,3321.39,3357.05
+'Women',2000,6,4950.270000,1956.92,2980.12,2196.47
+'Shoes',2000,6,4926.845000,1942.09,2679.25,3503.44
+'Women',2000,5,4606.794167,1639.46,2157.87,2583.06
+'Women',2000,1,4997.976667,2033.94,4136.88,3386.81
+'Music',2000,3,5212.355000,2256.79,2483.30,3203.46
+'Shoes',2000,3,5326.516667,2380.52,3557.54,2381.65
+'Shoes',2000,4,5326.516667,2381.65,2380.52,3185.08
+'Music',2000,6,5076.010833,2166.79,2636.43,2696.26
+'Shoes',2000,5,5213.916667,2308.61,3484.35,2631.62
+'Shoes',2000,7,4944.166667,2043.50,2714.17,7158.49
+'Children',2000,6,5109.577500,2231.87,2492.85,2716.69
+'Music',2000,7,5036.090833,2162.98,2169.49,7442.03
+'Music',2000,6,5036.090833,2169.49,3303.43,2162.98
+'Music',2000,3,4953.792500,2087.58,2881.98,2396.23
+'Shoes',2000,3,4824.920833,1981.97,2557.91,2178.72
+'Music',2000,4,5270.568333,2429.81,2500.28,3121.27
+'Women',2000,4,4390.827500,1562.28,2203.54,2814.98
+'Shoes',2000,2,4565.836667,1740.10,3299.55,2905.54
+'Women',2000,4,4950.270000,2125.36,3688.19,2980.12
+'Men',2000,7,4657.830000,1836.07,2432.42,6304.84
+'Shoes',2000,6,5094.271667,2285.80,2982.59,2944.67
+'Shoes',2000,5,5233.675000,2458.45,3530.35,4587.71
+'Music',2000,3,5270.568333,2500.28,3185.98,2429.81
+'Music',2000,3,5257.115000,2491.28,2494.71,2963.86
+'Music',2000,2,5257.115000,2494.71,2788.64,2491.28
+'Shoes',2000,4,5168.767500,2407.05,3240.95,2483.46
+'Women',2000,3,4422.386667,1666.68,2570.10,2097.53
+'Women',2000,7,4950.270000,2196.47,1956.92,7068.51
+'Men',2000,5,5016.401667,2267.73,2984.52,3051.68
+'Music',2000,4,5076.010833,2328.83,3139.50,2636.43
+'Men',2000,4,4657.830000,1925.05,2409.04,2999.06
+'Music',2000,2,4931.629167,2199.12,3942.13,2335.05
+'Music',2000,2,5212.355000,2483.30,4582.12,2256.79
+'Men',2000,3,4872.140833,2143.87,3353.30,2688.70
+'Music',2000,7,5212.355000,2487.88,2624.81,6788.15
+'Shoes',2000,7,4912.345000,2194.71,3320.16,7019.04
+'Women',2000,4,4521.217500,1803.95,2858.45,2451.32
+'Music',2000,7,5257.115000,2540.19,3558.30,7261.59
+'Women',2000,2,5183.971667,2471.71,3493.02,2920.36
+'Shoes',2000,3,4440.288333,1737.33,1857.98,2951.57
+'Women',2000,4,5167.868333,2468.50,3375.26,3236.87
+'Shoes',2000,2,4732.806667,2035.56,3540.84,2823.10
+'Music',2000,2,4960.848333,2267.45,3486.05,3329.26
+'Shoes',2000,2,5240.653333,2549.66,3081.94,2798.03
+'Shoes',2000,5,5168.767500,2483.46,2407.05,2983.37
+'Women',2000,5,4888.695833,2204.11,3149.43,2688.11
+'Music',2000,7,5270.568333,2587.41,3050.17,6970.69
+'Children',2000,7,4372.639167,1694.02,2257.72,6000.24
+'Music',2000,4,4916.410000,2240.65,3382.64,3182.24
+'Men',2000,6,4364.737500,1689.53,2992.20,3120.76
+'Shoes',2000,4,5090.508333,2415.99,3256.81,2925.27
+'Music',2000,4,4813.664167,2143.57,2637.02,2902.70
+'Men',2000,5,4595.062500,1926.36,2586.96,2922.71
+'Women',2000,4,4913.032500,2257.04,1779.75,2643.30
+'Men',2000,4,4371.782500,1718.21,2208.02,2721.25
+'Music',2000,2,5110.744167,2461.84,4401.88,2022.81
+'Women',2000,3,5192.789167,2544.73,3587.53,3197.85
+'Shoes',2000,4,4824.920833,2178.72,1981.97,2884.49
+'Shoes',2000,3,5233.675000,2588.01,3532.78,3530.35
+'Children',2000,5,4483.991667,1838.66,2619.28,2374.55
+'Music',2000,7,5043.647500,2403.81,3238.49,6835.65
+'Women',2000,7,4407.061667,1776.42,2981.25,5710.21
+'Men',2000,6,4830.246667,2199.80,2412.75,3006.00
+'Shoes',2000,6,5326.516667,2701.75,3185.08,3285.18
+'Music',2000,2,4916.410000,2298.33,3493.76,3382.64
+'Women',2000,5,4727.370000,2110.50,2519.62,2267.53
+'Children',2000,5,5109.577500,2492.85,2728.55,2231.87
+'Music',2000,4,5043.647500,2430.73,2644.32,3481.08
+'Music',2000,6,4131.199167,1519.23,2157.93,2655.31
+'Music',2000,2,5172.156667,2563.98,2905.94,3777.26
+'Music',2000,4,4931.629167,2327.08,2335.05,2882.04
+'Music',2000,5,4960.848333,2360.20,2406.42,3555.36
+'Shoes',2000,4,4823.300000,2222.80,3016.25,2462.91
+'Music',2000,3,4931.629167,2335.05,2199.12,2327.08
+'Music',2000,6,5212.355000,2624.81,2899.34,2487.88
+'Men',2000,3,5016.401667,2432.36,3088.82,2984.52
+'Shoes',2000,2,4440.288333,1857.98,3346.62,1737.33
+'Shoes',2000,6,5213.916667,2631.62,2308.61,3081.21
+'Music',2000,6,5000.816667,2419.42,3117.07,3086.00
+'Children',2000,3,4551.438333,1973.72,2083.41,2621.49
+'Music',2000,7,5212.675833,2638.50,3720.45,7180.32
+'Women',2000,2,5092.845000,2525.38,3066.60,2819.83
+'Shoes',2000,2,5213.916667,2649.38,3206.27,3351.30
+'Men',2000,1,4595.062500,2032.46,3557.44,2372.72
+'Shoes',2000,7,5032.343333,2469.75,2961.97,6219.49
+'Women',2000,6,4776.425000,2216.70,2861.21,3620.51
+'Women',2000,7,4727.370000,2168.76,2267.53,7819.76
+'Music',2000,4,4953.792500,2396.23,2087.58,2921.95
+'Children',2000,5,5206.238333,2649.03,2879.82,3447.77
+---- TYPES
+STRING, INT, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q48.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q48.test
new file mode 100644
index 0000000..c08e538
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q48.test
@@ -0,0 +1,71 @@
+====
+---- QUERY: TPCDS-Q48
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 1998
+ and
+ (
+  (
+   cd_demo_sk = ss_cdemo_sk
+   and
+   cd_marital_status = 'M'
+   and
+   cd_education_status = '4 yr Degree'
+   and
+   ss_sales_price between 100.00 and 150.00
+   )
+ or
+  (
+  cd_demo_sk = ss_cdemo_sk
+   and
+   cd_marital_status = 'M'
+   and
+   cd_education_status = '4 yr Degree'
+   and
+   ss_sales_price between 50.00 and 100.00
+  )
+ or
+ (
+  cd_demo_sk = ss_cdemo_sk
+  and
+   cd_marital_status = 'M'
+   and
+   cd_education_status = '4 yr Degree'
+   and
+   ss_sales_price between 150.00 and 200.00
+ )
+ )
+ and
+ (
+  (
+  ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 0 and 2000
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 3000
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 25000
+  )
+ )
+;
+---- RESULTS
+18742
+---- TYPES
+BIGINT
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q57.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q57.test
new file mode 100644
index 0000000..6719833
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q57.test
@@ -0,0 +1,151 @@
+====
+---- QUERY: TPCDS-Q57
+with v1 as(
+ select i_category, i_brand,
+        cc_name,
+        d_year, d_moy,
+        sum(cs_sales_price) sum_sales,
+        avg(sum(cs_sales_price)) over
+          (partition by i_category, i_brand,
+                     cc_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     cc_name
+           order by d_year, d_moy) rn
+ from item, catalog_sales, date_dim, call_center
+ where cs_item_sk = i_item_sk and
+       cs_sold_date_sk = d_date_sk and
+       cc_call_center_sk= cs_call_center_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          cc_name , d_year, d_moy),
+ v2 as(
+ select v1.i_category, v1.i_brand
+        ,v1.d_year, v1.d_moy
+        ,v1.avg_monthly_sales
+        ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1. cc_name = v1_lag. cc_name and
+       v1. cc_name = v1_lead. cc_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+  select  *
+ from v2
+ where  d_year = 2000 and
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, 3
+ limit 100;
+---- RESULTS
+'Music','exportischolar #2',2000,2,7201.093333,2787.29,3877.25,4135.16
+'Music','edu packscholar #2',2000,3,7565.594167,3181.66,3343.32,4565.20
+'Women','importoamalg #2',2000,4,7052.317500,2758.84,5323.46,3397.55
+'Music','edu packscholar #2',2000,2,7565.594167,3343.32,4935.00,3181.66
+'Children','edu packexporti #2',2000,2,6345.205000,2210.46,3973.84,3221.55
+'Shoes','amalgedu pack #2',2000,2,7209.571667,3085.07,3565.30,4091.47
+'Music','exportischolar #2',2000,4,7201.093333,3133.86,4135.16,4593.34
+'Music','edu packscholar #2',2000,6,7565.594167,3524.34,5698.93,3734.51
+'Women','amalgamalg #2',2000,4,6500.189167,2467.19,2614.22,4090.61
+'Music','importoscholar #2',2000,5,7209.978333,3179.64,4922.03,3375.16
+'Shoes','importoedu pack #2',2000,4,7799.847500,3779.41,5531.49,4100.04
+'Shoes','edu packedu pack #2',2000,3,6570.579167,2582.43,3361.18,4247.21
+'Women','amalgamalg #2',2000,5,6974.840833,2986.83,4369.56,4425.63
+'Music','exportischolar #2',2000,4,7189.194167,3209.91,4070.55,4736.81
+'Shoes','exportiedu pack #2',2000,7,6282.870000,2323.03,3618.89,8561.10
+'Shoes','importoedu pack #2',2000,5,7451.172500,3497.68,4029.18,3736.68
+'Shoes','amalgedu pack #2',2000,5,7209.571667,3288.79,3876.29,5356.13
+'Women','amalgamalg #2',2000,3,6500.189167,2614.22,4120.11,2467.19
+'Music','importoscholar #2',2000,6,7173.549167,3305.49,4124.85,5517.87
+'Music','importoscholar #2',2000,5,7333.867500,3476.24,4186.38,4310.57
+'Women','edu packamalg #2',2000,4,7077.025000,3221.39,4578.37,3950.64
+'Music','importoscholar #2',2000,6,7209.978333,3375.16,3179.64,4200.45
+'Music','edu packscholar #2',2000,7,7565.594167,3734.51,3524.34,10911.39
+'Music','edu packscholar #2',2000,4,7393.935000,3566.55,4683.15,4192.86
+'Children','importoexporti #2',2000,4,6736.021667,2938.80,4408.60,3459.57
+'Music','edu packscholar #2',2000,6,7393.935000,3605.67,4192.86,4584.42
+'Children','importoexporti #2',2000,5,7750.523333,3964.46,5069.64,5289.29
+'Children','importoexporti #2',2000,2,7750.523333,3970.89,4680.36,5186.78
+'Music','importoscholar #2',2000,1,7173.549167,3425.31,7158.98,3653.76
+'Shoes','importoedu pack #2',2000,6,7003.122500,3271.77,4315.53,3844.47
+'Shoes','amalgedu pack #2',2000,7,7547.715000,3817.97,4325.72,10447.92
+'Shoes','importoedu pack #2',2000,6,7451.172500,3736.68,3497.68,4623.33
+'Women','importoamalg #2',2000,7,7052.317500,3341.72,4560.01,10274.76
+'Children','importoexporti #2',2000,6,6736.021667,3027.47,3459.57,3790.37
+'Shoes','amalgedu pack #2',2000,1,7547.715000,3844.74,5202.59,4396.86
+'Shoes','importoedu pack #2',2000,5,7799.847500,4100.04,3779.41,4264.74
+'Shoes','amalgedu pack #2',2000,5,7547.715000,3869.87,5245.43,4325.72
+'Music','amalgscholar #2',2000,1,6166.312500,2490.48,7622.41,2511.57
+'Children','importoexporti #2',2000,3,7239.694167,3569.22,3791.71,3662.60
+'Music','exportischolar #2',2000,2,7180.318333,3517.49,4135.97,5162.68
+'Women','importoamalg #2',2000,5,7052.317500,3397.55,2758.84,4560.01
+'Music','amalgscholar #2',2000,2,6166.312500,2511.57,2490.48,4536.16
+'Men','importoimporto #2',2000,4,6884.735833,3238.62,4209.58,3839.59
+'Shoes','amalgedu pack #2',2000,1,7209.571667,3565.30,8102.54,3085.07
+'Children','importoexporti #2',2000,4,7239.694167,3662.60,3569.22,5525.78
+'Women','edu packamalg #2',2000,6,7358.001667,3787.73,5088.83,5803.55
+'Shoes','exportiedu pack #2',2000,1,6443.398333,2873.80,7194.15,3855.27
+'Music','exportischolar #2',2000,6,7180.318333,3612.29,5020.81,4207.36
+'Children','edu packexporti #2',2000,3,6527.414167,2961.33,4500.62,3803.47
+'Music','importoscholar #2',2000,1,7209.978333,3647.90,6746.00,4352.48
+'Music','exportischolar #2',2000,6,7189.194167,3629.32,4736.81,3981.89
+'Women','edu packamalg #2',2000,5,7291.365000,3736.99,4426.42,4092.15
+'Shoes','edu packedu pack #2',2000,5,6600.886667,3047.98,3510.15,3138.61
+'Shoes','importoedu pack #2',2000,6,7799.847500,4264.74,4100.04,4268.37
+'Shoes','importoedu pack #2',2000,7,7799.847500,4268.37,4264.74,10510.36
+'Shoes','importoedu pack #2',2000,2,7451.172500,3929.11,4573.75,4039.83
+'Music','importoscholar #2',2000,2,7173.549167,3653.76,3425.31,3947.95
+'Women','edu packamalg #2',2000,2,7077.025000,3564.97,5471.15,4578.37
+'Men','importoimporto #2',2000,4,6918.526667,3411.02,3709.06,4760.77
+'Women','importoamalg #2',2000,7,7334.011667,3849.94,4082.40,9171.27
+'Men','edu packimporto #2',2000,4,5950.797500,2470.54,3971.86,3035.58
+'Shoes','edu packedu pack #2',2000,6,6600.886667,3138.61,3047.98,4619.22
+'Shoes','amalgedu pack #2',2000,7,7209.571667,3749.49,5356.13,9885.01
+'Shoes','edu packedu pack #2',2000,1,6600.886667,3142.19,6834.22,3525.41
+'Children','importoexporti #2',2000,2,7239.694167,3791.71,5862.92,3569.22
+'Women','importoamalg #2',2000,4,7334.011667,3908.07,4004.54,4347.22
+'Shoes','importoedu pack #2',2000,4,7451.172500,4029.18,4039.83,3497.68
+'Shoes','edu packedu pack #2',2000,7,6573.143333,3153.26,4172.69,9299.79
+'Men','importoimporto #2',2000,6,6918.526667,3499.04,4760.77,3845.54
+'Children','edu packexporti #2',2000,4,6507.459167,3094.38,3556.30,4301.81
+'Women','edu packamalg #2',2000,2,7291.365000,3879.78,4986.31,4063.80
+'Shoes','importoedu pack #2',2000,3,7451.172500,4039.83,3929.11,4029.18
+'Women','exportiamalg #2',2000,1,6440.529167,3035.08,5330.23,3319.38
+'Women','importoamalg #2',2000,6,7114.527500,3731.58,4087.83,3968.11
+'Shoes','edu packedu pack #2',2000,4,6573.143333,3198.44,3446.93,3735.20
+'Shoes','exportiedu pack #2',2000,6,6323.389167,2950.42,3321.16,3547.45
+'Men','exportiimporto #2',2000,2,6155.525000,2805.25,3204.31,4158.65
+'Women','exportiamalg #2',2000,6,6567.906667,3219.25,4349.74,4169.37
+'Children','exportiexporti #2',2000,7,5789.609167,2442.48,3626.57,9336.54
+'Music','edu packscholar #2',2000,2,7393.935000,4054.26,4200.58,4683.15
+'Men','amalgimporto #2',2000,2,6097.260000,2760.55,3906.28,3409.11
+'Women','importoamalg #2',2000,1,7114.527500,3780.48,7156.48,4503.18
+'Shoes','amalgedu pack #2',2000,4,7209.571667,3876.29,4091.47,3288.79
+'Men','importoimporto #2',2000,7,6884.735833,3551.59,4528.39,8535.51
+'Women','importoamalg #2',2000,3,7334.011667,4004.54,4128.85,3908.07
+'Music','exportischolar #2',2000,1,7201.093333,3877.25,7562.58,2787.29
+'Shoes','exportiedu pack #2',2000,4,6282.870000,2962.87,4050.37,3568.63
+'Children','amalgexporti #2',2000,2,6735.918333,3434.61,4322.80,3825.99
+'Men','edu packimporto #2',2000,4,6081.163333,2781.12,4744.62,3539.63
+'Women','amalgamalg #2',2000,7,6974.840833,3676.16,4425.63,8099.82
+'Children','edu packexporti #2',2000,5,6527.414167,3239.36,3803.47,3355.26
+'Shoes','amalgedu pack #2',2000,5,6576.220000,3290.55,4295.49,3497.97
+'Music','importoscholar #2',2000,4,7173.549167,3896.97,3947.95,4124.85
+'Children','importoexporti #2',2000,5,6736.021667,3459.57,2938.80,3027.47
+'Men','amalgimporto #2',2000,4,5789.472500,2519.48,2917.13,3100.93
+'Children','amalgexporti #2',2000,7,5990.689167,2722.94,3144.36,8294.07
+'Women','exportiamalg #2',2000,4,6440.529167,3178.75,4514.55,4645.14
+'Women','amalgamalg #2',2000,1,6974.840833,3719.07,7297.36,4378.50
+'Children','importoexporti #2',2000,7,7750.523333,4494.79,5289.29,8873.39
+'Men','importoimporto #2',2000,2,6884.735833,3629.86,4031.03,4209.58
+---- TYPES
+STRING, STRING, INT, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q58.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q58.test
new file mode 100644
index 0000000..e36b55f
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q58.test
@@ -0,0 +1,68 @@
+====
+---- QUERY: TPCDS-Q58
+with ss_items as
+ (select i_item_id item_id
+        ,sum(ss_ext_sales_price) ss_item_rev
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk = i_item_sk
+   and d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+   and ss_sold_date_sk   = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+        ,sum(cs_ext_sales_price) cs_item_rev
+  from catalog_sales
+      ,item
+      ,date_dim
+ where cs_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+  and  cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+        ,sum(ws_ext_sales_price) ws_item_rev
+  from web_sales
+      ,item
+      ,date_dim
+ where ws_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq =(select d_week_seq
+                                     from date_dim
+                                     where d_date = '1998-02-19'))
+  and ws_sold_date_sk   = d_date_sk
+ group by i_item_id)
+  select  ss_items.item_id
+       ,ss_item_rev
+       ,ss_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ss_dev
+       ,cs_item_rev
+       ,cs_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 cs_dev
+       ,ws_item_rev
+       ,ws_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ws_dev
+       ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+   and ss_items.item_id=ws_items.item_id
+   and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+   and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+   and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+   and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+   and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+   and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ order by item_id
+         ,ss_item_rev
+ limit 100;
+---- RESULTS
+---- TYPES
+STRING, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q59.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q59.test
new file mode 100644
index 0000000..6778141
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q59.test
@@ -0,0 +1,147 @@
+====
+---- QUERY: TPCDS-Q59
+with wss as
+ (select d_week_seq,
+        ss_store_sk,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+  select  s_store_name1,s_store_id1,d_week_seq1
+       ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+       ,tue_sales1/tue_sales1,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+       ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+        ,s_store_id s_store_id1,sun_sales sun_sales1
+        ,mon_sales mon_sales1,tue_sales tue_sales1
+        ,wed_sales wed_sales1,thu_sales thu_sales1
+        ,fri_sales fri_sales1,sat_sales sat_sales1
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and
+        d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+        ,s_store_id s_store_id2,sun_sales sun_sales2
+        ,mon_sales mon_sales2,tue_sales tue_sales2
+        ,wed_sales wed_sales2,thu_sales thu_sales2
+        ,fri_sales fri_sales2,sat_sales sat_sales2
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and
+        d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+   and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100;
+---- RESULTS
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5153,0.724504,1.288464,1.000000,0.916377,1.087736,1.430693,1.056832
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5154,1.103366,0.896955,1.000000,1.156929,1.106482,0.896274,1.694469
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+'able','AAAAAAAACAAAAAAA',5155,1.095039,1.193688,1.000000,0.892349,1.132771,0.727840,1.034372
+---- TYPES
+STRING, STRING, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q63.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q63.test
new file mode 100644
index 0000000..6b958a0
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q63.test
@@ -0,0 +1,76 @@
+====
+---- QUERY: TPCDS-Q63
+select  *
+from (select i_manager_id
+             ,sum(ss_sales_price) sum_sales
+             ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+      from item
+          ,store_sales
+          ,date_dim
+          ,store
+      where ss_item_sk = i_item_sk
+        and ss_sold_date_sk = d_date_sk
+        and ss_store_sk = s_store_sk
+        and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+        and ((    i_category in ('Books','Children','Electronics')
+              and i_class in ('personal','portable','refernece','self-help')
+              and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+                                  'exportiunivamalg #9','scholaramalgamalg #9'))
+           or(    i_category in ('Women','Music','Men')
+              and i_class in ('accessories','classical','fragrances','pants')
+              and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+                                 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+        ,avg_monthly_sales
+        ,sum_sales
+limit 100;
+---- RESULTS
+10,58.29,219.285833
+10,86.13,219.285833
+10,99.79,219.285833
+10,103.41,219.285833
+10,123.23,219.285833
+10,145.67,219.285833
+10,180.92,219.285833
+10,246.39,219.285833
+10,439.43,219.285833
+10,457.26,219.285833
+10,480.87,219.285833
+13,11.61,170.405000
+13,11.82,170.405000
+13,71.49,170.405000
+13,75.64,170.405000
+13,84.31,170.405000
+13,113.69,170.405000
+13,253.69,170.405000
+13,291.33,170.405000
+13,304.78,170.405000
+13,461.69,170.405000
+29,4.08,171.758182
+29,18.16,171.758182
+29,44.87,171.758182
+29,71.17,171.758182
+29,102.90,171.758182
+29,138.61,171.758182
+29,208.57,171.758182
+29,247.06,171.758182
+29,248.03,171.758182
+29,372.54,171.758182
+29,433.35,171.758182
+34,5.95,175.055000
+34,13.06,175.055000
+34,32.31,175.055000
+34,76.42,175.055000
+34,82.49,175.055000
+34,112.40,175.055000
+34,126.59,175.055000
+34,210.32,175.055000
+34,219.27,175.055000
+34,258.47,175.055000
+34,409.37,175.055000
+34,554.01,175.055000
+---- TYPES
+INT, DECIMAL, DECIMAL
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q83.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q83.test
new file mode 100644
index 0000000..5efc9b9
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q83.test
@@ -0,0 +1,74 @@
+====
+---- QUERY: TPCDS-Q83
+with sr_items as
+ (select i_item_id item_id,
+        sum(sr_return_quantity) sr_item_qty
+ from store_returns,
+      item,
+      date_dim
+ where sr_item_sk = i_item_sk
+ and   d_date    in
+        (select d_date
+        from date_dim
+        where d_week_seq in
+                (select d_week_seq
+                from date_dim
+          where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   sr_returned_date_sk   = d_date_sk
+ group by i_item_id),
+ cr_items as
+ (select i_item_id item_id,
+        sum(cr_return_quantity) cr_item_qty
+ from catalog_returns,
+      item,
+      date_dim
+ where cr_item_sk = i_item_sk
+ and   d_date    in
+       (select d_date
+       from date_dim
+       where d_week_seq in
+               (select d_week_seq
+               from date_dim
+         where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and  cr_returned_date_sk   = d_date_sk
+ group by i_item_id),
+ wr_items as
+ (select i_item_id item_id,
+        sum(wr_return_quantity) wr_item_qty
+ from web_returns,
+      item,
+      date_dim
+ where wr_item_sk = i_item_sk
+ and   d_date    in
+        (select d_date
+        from date_dim
+        where d_week_seq in
+                (select d_week_seq
+                from date_dim
+                where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   wr_returned_date_sk   = d_date_sk
+ group by i_item_id)
+  select  sr_items.item_id
+       ,sr_item_qty
+       ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev
+       ,cr_item_qty
+       ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev
+       ,wr_item_qty
+       ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev
+       ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average
+ from sr_items
+     ,cr_items
+     ,wr_items
+ where sr_items.item_id=cr_items.item_id
+   and sr_items.item_id=wr_items.item_id
+ order by sr_items.item_id
+         ,sr_item_qty
+ limit 100;
+---- RESULTS
+'AAAAAAAAAHKAAAAA',24,11.267605633803,39,18.309859154930,8,3.755868544601,23.666667
+'AAAAAAAAIFNDAAAA',4,1.801801801802,10,4.504504504505,60,27.027027027027,24.666667
+'AAAAAAAAKJADAAAA',17,8.095238095238,51,24.285714285714,2,0.952380952381,23.333333
+'AAAAAAAAOAIBAAAA',40,13.605442176871,32,10.884353741497,26,8.843537414966,32.666667
+---- TYPES
+STRING, BIGINT, DECIMAL, BIGINT, DECIMAL, BIGINT, DECIMAL, DECIMAL
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q85.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q85.test
new file mode 100644
index 0000000..b80431c
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q85.test
@@ -0,0 +1,90 @@
+====
+---- QUERY: TPCDS-Q85
+select  substr(r_reason_desc,1,20)
+       ,avg(ws_quantity)
+       ,avg(wr_refunded_cash)
+       ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+      customer_demographics cd2, customer_address, date_dim, reason
+ where ws_web_page_sk = wp_web_page_sk
+   and ws_item_sk = wr_item_sk
+   and ws_order_number = wr_order_number
+   and ws_sold_date_sk = d_date_sk and d_year = 1998
+   and cd1.cd_demo_sk = wr_refunded_cdemo_sk
+   and cd2.cd_demo_sk = wr_returning_cdemo_sk
+   and ca_address_sk = wr_refunded_addr_sk
+   and r_reason_sk = wr_reason_sk
+   and
+   (
+    (
+     cd1.cd_marital_status = 'M'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = '4 yr Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 100.00 and 150.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'D'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Primary'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 50.00 and 100.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'U'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Advanced Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 150.00 and 200.00
+    )
+   )
+   and
+   (
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('KY', 'GA', 'NM')
+     and ws_net_profit between 100 and 200
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('MT', 'OR', 'IN')
+     and ws_net_profit between 150 and 300
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('WI', 'MO', 'WV')
+     and ws_net_profit between 50 and 250
+    )
+   )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+        ,avg(ws_quantity)
+        ,avg(wr_refunded_cash)
+        ,avg(wr_fee)
+limit 100;
+---- RESULTS
+'Did not like the mak',31.0,50.210000,95.170000
+'Found a better price',7.0,11.860000,27.890000
+'Not the product that',5.0,107.380000,69.700000
+---- TYPES
+STRING, DOUBLE, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q89.test b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q89.test
new file mode 100644
index 0000000..12c07d7
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q89.test
@@ -0,0 +1,131 @@
+====
+---- QUERY: TPCDS-Q89
+select  *
+from(
+select i_category, i_class, i_brand,
+       s_store_name, s_company_name,
+       d_moy,
+       sum(ss_sales_price) sum_sales,
+       avg(sum(ss_sales_price)) over
+         (partition by i_category, i_brand, s_store_name, s_company_name)
+         avg_monthly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
+      d_year in (2000) and
+        ((i_category in ('Home','Books','Electronics') and
+          i_class in ('wallpaper','parenting','musical')
+         )
+      or (i_category in ('Shoes','Jewelry','Men') and
+          i_class in ('womens','birdal','pants')
+        ))
+group by i_category, i_class, i_brand,
+         s_store_name, s_company_name, d_moy) tmp1
+where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1
+order by sum_sales - avg_monthly_sales, s_store_name
+limit 100;
+---- RESULTS
+'Shoes','womens','amalgedu pack #2','eing','Unknown',7,2043.50,4944.166667
+'Shoes','womens','amalgedu pack #2','able','Unknown',2,2035.56,4732.806667
+'Men','pants','exportiimporto #2','bar','Unknown',6,1689.53,4364.737500
+'Men','pants','exportiimporto #2','eing','Unknown',4,1783.84,4269.094167
+'Shoes','womens','amalgedu pack #2','able','Unknown',5,2250.32,4732.806667
+'Shoes','womens','amalgedu pack #2','ation','Unknown',6,2271.73,4711.609167
+'Shoes','womens','amalgedu pack #2','eing','Unknown',4,2516.16,4944.166667
+'Shoes','womens','amalgedu pack #2','ese','Unknown',2,2212.92,4631.164167
+'Shoes','womens','amalgedu pack #2','ation','Unknown',7,2293.91,4711.609167
+'Men','pants','exportiimporto #2','able','Unknown',4,2114.49,4471.488333
+'Shoes','womens','amalgedu pack #2','ese','Unknown',4,2335.75,4631.164167
+'Shoes','womens','amalgedu pack #2','able','Unknown',4,2471.45,4732.806667
+'Shoes','womens','amalgedu pack #2','eing','Unknown',6,2714.17,4944.166667
+'Shoes','womens','amalgedu pack #2','ought','Unknown',7,2385.30,4582.193333
+'Men','pants','exportiimporto #2','ese','Unknown',2,2002.22,4191.219167
+'Men','pants','exportiimporto #2','ought','Unknown',1,2013.92,4160.935833
+'Men','pants','exportiimporto #2','ese','Unknown',6,2045.85,4191.219167
+'Shoes','womens','amalgedu pack #2','eing','Unknown',3,2818.29,4944.166667
+'Shoes','womens','amalgedu pack #2','ought','Unknown',2,2484.68,4582.193333
+'Shoes','womens','amalgedu pack #2','ation','Unknown',3,2617.58,4711.609167
+'Shoes','womens','amalgedu pack #2','ought','Unknown',4,2489.01,4582.193333
+'Shoes','womens','amalgedu pack #2','bar','Unknown',7,2473.32,4563.120833
+'Men','pants','exportiimporto #2','bar','Unknown',3,2284.67,4364.737500
+'Shoes','womens','amalgedu pack #2','able','Unknown',6,2661.93,4732.806667
+'Shoes','womens','amalgedu pack #2','bar','Unknown',2,2518.10,4563.120833
+'Men','pants','exportiimporto #2','eing','Unknown',2,2228.12,4269.094167
+'Men','pants','exportiimporto #2','ought','Unknown',5,2123.43,4160.935833
+'Men','pants','exportiimporto #2','able','Unknown',3,2442.59,4471.488333
+'Shoes','womens','amalgedu pack #2','eing','Unknown',2,2943.53,4944.166667
+'Shoes','womens','amalgedu pack #2','ation','Unknown',4,2730.48,4711.609167
+'Men','pants','exportiimporto #2','ought','Unknown',4,2191.09,4160.935833
+'Men','pants','exportiimporto #2','eing','Unknown',5,2303.47,4269.094167
+'Shoes','womens','amalgedu pack #2','ought','Unknown',3,2636.91,4582.193333
+'Shoes','womens','amalgedu pack #2','ation','Unknown',5,2780.52,4711.609167
+'Men','pants','exportiimporto #2','ation','Unknown',4,2225.47,4154.930000
+'Shoes','womens','amalgedu pack #2','ese','Unknown',6,2703.10,4631.164167
+'Men','pants','exportiimporto #2','ation','Unknown',2,2241.65,4154.930000
+'Shoes','womens','amalgedu pack #2','able','Unknown',3,2823.10,4732.806667
+'Shoes','womens','amalgedu pack #1','ation','Unknown',2,752.99,2646.467500
+'Men','pants','exportiimporto #2','able','Unknown',5,2601.64,4471.488333
+'Shoes','womens','amalgedu pack #2','able','Unknown',7,2864.95,4732.806667
+'Shoes','womens','amalgedu pack #2','ese','Unknown',3,2768.79,4631.164167
+'Men','pants','exportiimporto #2','ese','Unknown',5,2401.63,4191.219167
+'Shoes','womens','amalgedu pack #2','bar','Unknown',6,2799.28,4563.120833
+'Men','pants','exportiimporto #2','ation','Unknown',5,2421.85,4154.930000
+'Shoes','womens','amalgedu pack #2','ought','Unknown',6,2853.29,4582.193333
+'Men','pants','exportiimporto #2','ought','Unknown',2,2432.48,4160.935833
+'Men','pants','exportiimporto #2','able','Unknown',6,2760.92,4471.488333
+'Men','pants','exportiimporto #2','eing','Unknown',6,2559.62,4269.094167
+'Men','pants','exportiimporto #2','ation','Unknown',3,2453.59,4154.930000
+'Shoes','womens','amalgedu pack #2','bar','Unknown',4,2950.74,4563.120833
+'Men','pants','exportiimporto #2','ation','Unknown',1,2566.01,4154.930000
+'Shoes','womens','amalgedu pack #2','eing','Unknown',1,3373.13,4944.166667
+'Shoes','womens','amalgedu pack #1','bar','Unknown',2,746.66,2311.219167
+'Men','pants','exportiimporto #2','ese','Unknown',7,2631.49,4191.219167
+'Shoes','womens','amalgedu pack #2','ation','Unknown',2,3154.14,4711.609167
+'Men','pants','exportiimporto #2','able','Unknown',1,2918.84,4471.488333
+'Shoes','womens','amalgedu pack #2','ese','Unknown',1,3089.25,4631.164167
+'Shoes','womens','amalgedu pack #2','bar','Unknown',3,3050.52,4563.120833
+'Men','pants','exportiimporto #1','eing','Unknown',4,583.03,2095.400833
+'Shoes','womens','amalgedu pack #1','eing','Unknown',6,991.32,2497.713333
+'Men','pants','exportiimporto #2','ation','Unknown',6,2649.32,4154.930000
+'Men','pants','exportiimporto #2','able','Unknown',7,2973.59,4471.488333
+'Shoes','womens','amalgedu pack #1','bar','Unknown',3,828.20,2311.219167
+'Men','pants','exportiimporto #2','ought','Unknown',7,2701.30,4160.935833
+'Shoes','womens','amalgedu pack #1','eing','Unknown',4,1049.89,2497.713333
+'Shoes','womens','amalgedu pack #2','bar','Unknown',5,3126.30,4563.120833
+'Shoes','womens','amalgedu pack #1','ation','Unknown',7,1230.33,2646.467500
+'Men','pants','exportiimporto #2','ation','Unknown',7,2739.09,4154.930000
+'Shoes','womens','amalgedu pack #2','ought','Unknown',5,3167.99,4582.193333
+'Men','pants','exportiimporto #2','bar','Unknown',4,2977.49,4364.737500
+'Shoes','womens','amalgedu pack #2','eing','Unknown',5,3557.58,4944.166667
+'Men','pants','exportiimporto #2','bar','Unknown',5,2992.20,4364.737500
+'Men','pants','exportiimporto #2','able','Unknown',2,3099.75,4471.488333
+'Shoes','womens','amalgedu pack #2','ought','Unknown',1,3214.28,4582.193333
+'Men','pants','exportiimporto #2','ought','Unknown',3,2795.06,4160.935833
+'Shoes','womens','amalgedu pack #1','able','Unknown',6,1155.46,2507.540833
+'Shoes','womens','amalgedu pack #2','bar','Unknown',1,3217.46,4563.120833
+'Shoes','womens','amalgedu pack #1','ought','Unknown',7,1142.24,2459.560000
+'Shoes','womens','amalgedu pack #1','able','Unknown',2,1203.83,2507.540833
+'Men','pants','exportiimporto #1','eing','Unknown',7,797.62,2095.400833
+'Men','pants','exportiimporto #2','eing','Unknown',1,2980.60,4269.094167
+'Shoes','womens','amalgedu pack #1','ation','Unknown',4,1359.31,2646.467500
+'Shoes','womens','amalgedu pack #1','ought','Unknown',2,1174.72,2459.560000
+'Men','pants','exportiimporto #1','bar','Unknown',5,671.59,1948.488333
+'Men','pants','exportiimporto #2','bar','Unknown',2,3099.19,4364.737500
+'Shoes','womens','amalgedu pack #1','able','Unknown',3,1248.26,2507.540833
+'Men','pants','exportiimporto #2','bar','Unknown',7,3120.76,4364.737500
+'Shoes','womens','amalgedu pack #1','ation','Unknown',6,1407.42,2646.467500
+'Shoes','womens','amalgedu pack #1','ese','Unknown',4,1285.46,2520.399167
+'Men','pants','exportiimporto #2','ese','Unknown',1,2962.47,4191.219167
+'Shoes','womens','amalgedu pack #1','ought','Unknown',4,1248.31,2459.560000
+'Shoes','womens','amalgedu pack #2','ation','Unknown',1,3507.39,4711.609167
+'Shoes','womens','amalgedu pack #2','able','Unknown',1,3540.84,4732.806667
+'Men','pants','exportiimporto #1','ese','Unknown',4,720.85,1911.922500
+'Men','pants','exportiimporto #1','able','Unknown',6,741.13,1929.061667
+'Men','pants','exportiimporto #2','ought','Unknown',6,2973.45,4160.935833
+'Shoes','womens','amalgedu pack #1','able','Unknown',7,1326.81,2507.540833
+'Shoes','womens','amalgedu pack #1','ought','Unknown',3,1290.11,2459.560000
+'Men','pants','exportiimporto #2','bar','Unknown',1,3201.04,4364.737500
+---- TYPES
+STRING, STRING, STRING, STRING, STRING, INT, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q26.test b/testdata/workloads/tpcds/queries/tpcds-q26.test
new file mode 100644
index 0000000..b75702b
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q26.test
@@ -0,0 +1,124 @@
+====
+---- QUERY: TPCDS-Q26
+select  i_item_id,
+        avg(cs_quantity) agg1,
+        avg(cs_list_price) agg2,
+        avg(cs_coupon_amt) agg3,
+        avg(cs_sales_price) agg4
+ from catalog_sales, customer_demographics, date_dim, item, promotion
+ where cs_sold_date_sk = d_date_sk and
+       cs_item_sk = i_item_sk and
+       cs_bill_cdemo_sk = cd_demo_sk and
+       cs_promo_sk = p_promo_sk and
+       cd_gender = 'F' and
+       cd_marital_status = 'W' and
+       cd_education_status = 'Primary' and
+       (p_channel_email = 'N' or p_channel_event = 'N') and
+       d_year = 1998
+ group by i_item_id
+ order by i_item_id
+ limit 100;
+---- RESULTS
+'AAAAAAAAAAACAAAA',76,154.00,217.69,9.24
+'AAAAAAAAAABAAAAA',8,52.11,0.00,44.29
+'AAAAAAAAAABBAAAA',53,35.02,0.00,3.50
+'AAAAAAAAAACDAAAA',28,153.65,0.00,136.74
+'AAAAAAAAAADBAAAA',9.5,43.05,0.00,10.19
+'AAAAAAAAAADEAAAA',70,35.22,0.00,26.41
+'AAAAAAAAAAEAAAAA',26,119.48,0.00,24.73
+'AAAAAAAAAAFCAAAA',20,199.23,0.00,27.89
+'AAAAAAAAAAHAAAAA',17,39.42,0.00,12.22
+'AAAAAAAAAAHBAAAA',72,81.19,0.00,68.19
+'AAAAAAAAAAIAAAAA',44,76.98,0.00,7.69
+'AAAAAAAAAAICAAAA',70,189.36,0.00,176.10
+'AAAAAAAAAAIDAAAA',12,68.97,0.00,66.21
+'AAAAAAAAAAKBAAAA',24,146.99,0.00,70.55
+'AAAAAAAAAAKDAAAA',98,35.03,0.00,14.36
+'AAAAAAAAAALAAAAA',45,152.86,1232.64,85.60
+'AAAAAAAAAALDAAAA',8,148.55,115.27,144.09
+'AAAAAAAAAAOAAAAA',16,58.17,0.00,48.28
+'AAAAAAAAAAODAAAA',81,97.91,746.79,69.14
+'AAAAAAAAAAPBAAAA',42,104.44,0.00,97.12
+'AAAAAAAAAAPCAAAA',11,77.94,0.00,18.70
+'AAAAAAAAABAAAAAA',55,121.15,320.54,76.12
+'AAAAAAAAABADAAAA',42.5,180.66,0.00,131.27
+'AAAAAAAAABAEAAAA',49,169.48,0.00,27.59
+'AAAAAAAAABBAAAAA',51,29.47,0.00,13.55
+'AAAAAAAAABBDAAAA',14,15.10,34.12,12.83
+'AAAAAAAAABCBAAAA',74,191.51,0.00,61.28
+'AAAAAAAAABDAAAAA',80,69.26,959.43,29.44
+'AAAAAAAAABDBAAAA',97,65.13,2430.47,33.86
+'AAAAAAAAABFBAAAA',16,86.31,0.00,63.86
+'AAAAAAAAABFCAAAA',31,1.37,0.00,1.13
+'AAAAAAAAABGDAAAA',47,165.52,293.28,48.00
+'AAAAAAAAABHDAAAA',61,177.80,690.75,23.11
+'AAAAAAAAABIBAAAA',72,139.09,0.00,27.81
+'AAAAAAAAABJAAAAA',45,36.29,0.00,0.00
+'AAAAAAAAABKAAAAA',77,107.03,0.00,48.16
+'AAAAAAAAABKDAAAA',100,16.45,0.00,4.60
+'AAAAAAAAABLCAAAA',1,3.43,0.00,1.74
+'AAAAAAAAABNAAAAA',54.5,68.89,0.00,21.38
+'AAAAAAAAABNCAAAA',43.5,120.58,142.81,73.43
+'AAAAAAAAABOBAAAA',13,63.93,0.00,0.00
+'AAAAAAAAABPAAAAA',28,53.25,0.00,13.31
+'AAAAAAAAABPDAAAA',40.25,142.67,58.73,47.19
+'AAAAAAAAACBBAAAA',94,69.53,0.00,52.14
+'AAAAAAAAACBCAAAA',94,63.22,0.00,34.13
+'AAAAAAAAACCAAAAA',66,17.10,0.00,16.92
+'AAAAAAAAACCBAAAA',62,106.89,0.00,16.03
+'AAAAAAAAACCDAAAA',34,103.07,0.00,51.53
+'AAAAAAAAACDAAAAA',46.5,170.48,733.16,60.52
+'AAAAAAAAACDCAAAA',28,162.28,0.00,108.72
+'AAAAAAAAACFBAAAA',4,2.01,0.00,0.94
+'AAAAAAAAACFEAAAA',23,67.39,0.00,43.12
+'AAAAAAAAACGAAAAA',25,111.47,0.00,84.71
+'AAAAAAAAACGCAAAA',34,131.03,0.00,74.86
+'AAAAAAAAACGDAAAA',44.5,57.51,0.00,11.75
+'AAAAAAAAACIAAAAA',68.33333333333333,112.60,0.00,57.82
+'AAAAAAAAACKBAAAA',78,84.74,679.46,26.35
+'AAAAAAAAACKCAAAA',51.5,133.95,0.00,109.20
+'AAAAAAAAACLAAAAA',58.5,127.34,9.22,11.57
+'AAAAAAAAACLBAAAA',86,39.73,0.00,3.97
+'AAAAAAAAACMAAAAA',68,10.30,0.00,2.47
+'AAAAAAAAACOBAAAA',42.5,14.95,0.00,14.00
+'AAAAAAAAACODAAAA',68,40.41,0.00,9.29
+'AAAAAAAAADBAAAAA',55.5,68.33,25.58,63.21
+'AAAAAAAAADBDAAAA',69,25.33,0.00,9.37
+'AAAAAAAAADCCAAAA',48,100.58,0.00,28.16
+'AAAAAAAAADEAAAAA',44.5,94.81,955.39,51.70
+'AAAAAAAAADEBAAAA',48,47.45,0.00,3.32
+'AAAAAAAAADEDAAAA',71,150.09,0.00,112.56
+'AAAAAAAAADEEAAAA',36,66.39,0.00,17.92
+'AAAAAAAAADFAAAAA',21.5,115.71,1641.00,102.13
+'AAAAAAAAADFCAAAA',32.5,113.63,0.00,67.83
+'AAAAAAAAADFDAAAA',32,47.50,0.00,23.75
+'AAAAAAAAADHAAAAA',79,228.17,0.00,73.01
+'AAAAAAAAADICAAAA',74,131.62,0.00,101.34
+'AAAAAAAAADKDAAAA',1,127.88,83.24,89.51
+'AAAAAAAAADLCAAAA',84,27.84,247.97,18.62
+'AAAAAAAAADNAAAAA',56,207.02,0.00,163.54
+'AAAAAAAAADNBAAAA',81,21.23,0.00,0.84
+'AAAAAAAAADOAAAAA',32,15.37,0.00,6.33
+'AAAAAAAAADPBAAAA',5,49.32,0.00,8.87
+'AAAAAAAAAEADAAAA',68,26.81,0.00,26.00
+'AAAAAAAAAEBAAAAA',34,206.22,0.00,181.47
+'AAAAAAAAAECEAAAA',42,177.93,0.00,86.52
+'AAAAAAAAAEDAAAAA',5,66.02,304.04,62.05
+'AAAAAAAAAEDDAAAA',73,192.70,0.00,21.19
+'AAAAAAAAAEECAAAA',84,93.98,0.00,93.04
+'AAAAAAAAAEGBAAAA',25,221.44,0.00,24.35
+'AAAAAAAAAEHAAAAA',49,3.28,0.00,3.24
+'AAAAAAAAAEHCAAAA',10,66.10,0.00,46.93
+'AAAAAAAAAEHDAAAA',68,88.66,0.00,88.66
+'AAAAAAAAAEJBAAAA',48,228.35,0.00,150.71
+'AAAAAAAAAEJDAAAA',51,102.02,0.00,4.08
+'AAAAAAAAAEKAAAAA',88,50.12,527.15,11.52
+'AAAAAAAAAEKCAAAA',51.75,123.51,742.54,85.63
+'AAAAAAAAAELBAAAA',95,8.37,0.00,7.70
+'AAAAAAAAAEMBAAAA',13,120.05,340.80,33.61
+'AAAAAAAAAENCAAAA',90,81.43,0.00,77.35
+'AAAAAAAAAEOBAAAA',86,92.17,0.00,0.00
+'AAAAAAAAAEOCAAAA',76,40.17,0.00,28.52
+---- TYPES
+STRING, DOUBLE, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q30.test b/testdata/workloads/tpcds/queries/tpcds-q30.test
new file mode 100644
index 0000000..0d5f5eb
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q30.test
@@ -0,0 +1,126 @@
+====
+---- QUERY: TPCDS-Q30
+with customer_total_return as
+ (select wr_returning_customer_sk as ctr_customer_sk
+        ,ca_state as ctr_state,
+    sum(wr_return_amt) as ctr_total_return
+ from web_returns
+     ,date_dim
+     ,customer_address
+ where wr_returned_date_sk = d_date_sk
+   and d_year =2002
+   and wr_returning_addr_sk = ca_address_sk
+ group by wr_returning_customer_sk
+         ,ca_state)
+  select  c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+       ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+       ,c_last_review_date,ctr_total_return
+ from customer_total_return ctr1
+     ,customer_address
+     ,customer
+ where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2
+              from customer_total_return ctr2
+                          where ctr1.ctr_state = ctr2.ctr_state)
+       and ca_address_sk = c_current_addr_sk
+       and ca_state = 'IL'
+       and ctr1.ctr_customer_sk = c_customer_sk
+ order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag
+                  ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address
+                  ,c_last_review_date,ctr_total_return
+limit 100;
+---- RESULTS
+'AAAAAAAAACBFAAAA','Dr.','Clifton','Carr','N',4,4,1950,'INDIA','NULL','Clifton.Carr@mOm0RG.com','2452385',2232.42
+'AAAAAAAAADCCAAAA','Sir','Elmer','Southerland','Y',14,3,1967,'NICARAGUA','NULL','Elmer.Southerland@dG9HZHGjU8.edu','2452623',9604.53
+'AAAAAAAAADFDBAAA','Dr.','Stanley','Smith','N',23,4,1928,'SPAIN','NULL','Stanley.Smith@Yxk9F0obDmkvN.org','2452381',2287.41
+'AAAAAAAAAEHLAAAA','Ms.','Anita','Vance','Y',14,3,1988,'ZIMBABWE','NULL','Anita.Vance@SCZrSvm7MpKZ.org','2452362',4263.60
+'AAAAAAAAAIJOAAAA','Dr.','Elise','Cook','Y',25,1,1957,'SAN MARINO','NULL','Elise.Cook@zgV4U.com','2452537',1940.00
+'AAAAAAAAAJLEBAAA','Mrs.','Judy','Wells','NULL',31,8,NULL,'JERSEY','NULL','NULL','NULL',1764.00
+'AAAAAAAAAMHMAAAA','Mrs.','Angel','Farrington','N',5,5,1992,'NICARAGUA','NULL','Angel.Farrington@ESvsPDBzslNpvaZ.org','2452593',3722.86
+'AAAAAAAABEGGAAAA','Ms.','Linda','Bishop','N',25,5,1945,'ANDORRA','NULL','Linda.Bishop@SsgvULplErri.org','2452369',5049.28
+'AAAAAAAABLPMAAAA','Sir','Peter','Hogue','N',24,4,1979,'FRENCH GUIANA','NULL','Peter.Hogue@XZhvIIKsznzvj.com','2452640',1582.00
+'AAAAAAAABNBKAAAA','Dr.','Chelsea','Wellman','Y',5,2,1989,'NIGERIA','NULL','Chelsea.Wellman@RY0lIpgj89SBPI.com','2452454',4034.24
+'AAAAAAAABNECAAAA','Dr.','Thomas','Hollingsworth','N',30,9,1960,'AFGHANISTAN','NULL','Thomas.Hollingsworth@GM6CNHy.org','2452581',6428.07
+'AAAAAAAABNMAAAAA','Miss','Martha','Moore','N',5,7,1985,'BOUVET ISLAND','NULL','Martha.Moore@s.org','2452584',7708.40
+'AAAAAAAABNMEBAAA','Dr.','Sara','Mcmahon','N',23,10,1926,'BARBADOS','NULL','Sara.Mcmahon@DGmEehYquCX.edu','2452584',1290.24
+'AAAAAAAACDKGAAAA','Dr.','Martha','Mason','N',20,6,1970,'MOROCCO','NULL','Martha.Mason@F.org','2452359',3861.33
+'AAAAAAAACEONAAAA','Ms.','Sandra','Lopez','Y',31,3,1979,'EGYPT','NULL','Sandra.Lopez@pJc3Or8NbSu8MX.com','2452311',4688.40
+'AAAAAAAACGBPAAAA','Ms.','Florence','Cabrera','Y',30,6,1934,'GHANA','NULL','Florence.Cabrera@cm.com','2452610',1608.64
+'AAAAAAAACIGIBAAA','Dr.','Charles','Lopez','Y',11,10,1979,'URUGUAY','NULL','Charles.Lopez@gOyo.com','2452424',2025.00
+'AAAAAAAACIOGBAAA','Dr.','Gregorio','Lee','N',3,10,1970,'LESOTHO','NULL','Gregorio.Lee@iteZa836XrfF.edu','2452438',1590.30
+'AAAAAAAACJLIAAAA','Mr.','Philip','Kinder','N',1,10,1987,'BOLIVIA','NULL','Philip.Kinder@t4B5veuzCR390MP.com','2452595',2526.44
+'AAAAAAAACKDBAAAA','Miss','Kathy','Kelly','N',15,9,1988,'ANTIGUA AND BARBUDA','NULL','Kathy.Kelly@o.org','2452339',2719.92
+'AAAAAAAACPLIAAAA','Mr.','John','Williams','Y',11,9,1960,'COMOROS','NULL','John.Williams@Gn.com','2452392',4542.08
+'AAAAAAAACPOIAAAA','Dr.','Melvin','Phillips','Y',16,2,1987,'LESOTHO','NULL','Melvin.Phillips@MinR9.com','2452402',2647.26
+'AAAAAAAADADNAAAA','Dr.','Willie','Vitale','Y',30,11,1978,'SLOVAKIA','NULL','Willie.Vitale@rUnNJoaskyn6aLA.edu','2452642',6185.28
+'AAAAAAAADAGBAAAA','Ms.','Helen','Barker','N',10,5,1933,'ZAMBIA','NULL','Helen.Barker@V8ZT4ncYBuGSK.com','2452378',2791.32
+'AAAAAAAADNLLAAAA','Mrs.','Mae','Jackson','N',15,8,1948,'SYRIAN ARAB REPUBLIC','NULL','Mae.Jackson@Eh2TmyLnCy.org','2452379',5097.08
+'AAAAAAAADPCIAAAA','Miss','Wilma','Sanborn','N',15,6,1952,'BURUNDI','NULL','Wilma.Sanborn@2jbbN0ubazbL.org','2452333',2457.60
+'AAAAAAAAEHFFBAAA','Dr.','Michelle','Reed','N',1,3,1982,'ZAMBIA','NULL','Michelle.Reed@9yNA4PLMEjB.edu','2452546',5864.34
+'AAAAAAAAFABKAAAA','Dr.','Nancy','Duke','N',10,1,1961,'ERITREA','NULL','Nancy.Duke@h.edu','2452379',1607.04
+'AAAAAAAAFCKHBAAA','Mrs.','Helen','Ramsay','N',6,1,1971,'LIECHTENSTEIN','NULL','Helen.Ramsay@ZC1VJrVtx7It3Iz.com','2452626',2868.48
+'AAAAAAAAFGBABAAA','Mrs.','Louise','Douglas','N',12,10,1944,'ARUBA','NULL','Louise.Douglas@VzomaEVuSe.edu','2452630',9067.96
+'AAAAAAAAFPAIBAAA','Dr.','Michael','Roberge','N',29,1,1959,'VIET NAM','NULL','Michael.Roberge@YTN9dUXCpJmz5.org','2452378',4120.84
+'AAAAAAAAGAMHAAAA','Miss','Clementine','Darby','N',5,2,1952,'SAN MARINO','NULL','Clementine.Darby@8HfgoxcVueO.com','2452573',2829.50
+'AAAAAAAAGBJKAAAA','Dr.','Amy','Baker','Y',5,6,1970,'SAUDI ARABIA','NULL','Amy.Baker@U8marrVO5XN9oMK.com','2452370',2981.21
+'AAAAAAAAGDFJAAAA','Ms.','Margaret','Edmonds','NULL',24,NULL,NULL,'ARUBA','NULL','NULL','NULL',3097.44
+'AAAAAAAAGHKPAAAA','Miss','Kimberly','Clark','N',15,11,1937,'TONGA','NULL','Kimberly.Clark@xYl6FQTOMkQPi.edu','2452477',1852.52
+'AAAAAAAAGKPGAAAA','Mrs.','Rachel','Reed','Y',13,3,1950,'MALAWI','NULL','Rachel.Reed@x.com','2452574',2254.35
+'AAAAAAAAGNKPAAAA','Dr.','Marcus','Davis','N',12,2,1968,'KENYA','NULL','Marcus.Davis@ksKq.com','2452633',2388.68
+'AAAAAAAAGNKPAAAA','Dr.','Marcus','Davis','N',12,2,1968,'KENYA','NULL','Marcus.Davis@ksKq.com','2452633',5162.50
+'AAAAAAAAGPBFAAAA','Dr.','Lorenzo','Grace','Y',5,2,1984,'UNITED STATES','NULL','Lorenzo.Grace@M8CTliq9t0H6.edu','2452613',2904.66
+'AAAAAAAAHDBLAAAA','Mr.','Michael','Dupre','Y',5,9,1961,'TIMOR-LESTE','NULL','Michael.Dupre@AhCfsll84OUMVNZ.com','2452438',3593.72
+'AAAAAAAAHDKHBAAA','Sir','NULL','Free','N',6,NULL,1956,'NULL','NULL','NULL','2452293',2100.64
+'AAAAAAAAHEFEAAAA','Dr.','James','Martin','Y',28,9,1963,'PERU','NULL','James.Martin@31H8tf1eym.org','2452321',2928.24
+'AAAAAAAAHKCEBAAA','Mrs.','Stella','Osorio','N',2,10,1966,'ANGUILLA','NULL','Stella.Osorio@6M8UcU9r.org','2452421',2095.08
+'AAAAAAAAHNIFBAAA','Mrs.','Jodi','Hickman','N',15,9,1973,'MALAYSIA','NULL','Jodi.Hickman@OdAHHq4.com','2452312',8251.12
+'AAAAAAAAIFNHBAAA','Sir','Victor','Mccloskey','N',24,2,1940,'MONTSERRAT','NULL','Victor.Mccloskey@RymmApu9aLZ.com','2452604',1819.48
+'AAAAAAAAIGJGAAAA','Sir','Edward','Britton','Y',NULL,NULL,1981,'NULL','NULL','Edward.Britton@G.edu','NULL',3791.04
+'AAAAAAAAIHLGAAAA','Miss','Jennifer','Smith','N',31,10,1969,'TRINIDAD AND TOBAGO','NULL','Jennifer.Smith@vYc7AUOo7.org','2452564',7360.74
+'AAAAAAAAIIABAAAA','Miss','Rosa','Phillips','Y',19,3,1949,'SURINAME','NULL','Rosa.Phillips@ECXsk.com','2452488',6261.20
+'AAAAAAAAIIHMAAAA','Mr.','Ricky','Gerber','N',10,11,1973,'WALLIS AND FUTUNA','NULL','Ricky.Gerber@5dsb4vhLdmZm.edu','2452402',1707.12
+'AAAAAAAAIKABAAAA','Dr.','James','Taylor','Y',2,1,1967,'LEBANON','NULL','James.Taylor@eVoZuvYcqACN.org','2452537',5759.52
+'AAAAAAAAIMNPAAAA','Dr.','John','Mendoza','Y',7,1,1940,'HAITI','NULL','John.Mendoza@XDX7Ku8gVsR1LHn.org','2452462',3580.20
+'AAAAAAAAIMPAAAAA','Ms.','Lois','Washington','N',21,8,1973,'TONGA','NULL','Lois.Washington@uvUAMB9DuxcK.edu','2452319',1756.80
+'AAAAAAAAJAMHBAAA','Miss','Kathleen','Dupre','Y',7,10,1951,'TURKEY','NULL','Kathleen.Dupre@hfQ.edu','2452349',6127.60
+'AAAAAAAAJEIFBAAA','NULL','NULL','NULL','N',NULL,NULL,1967,'NULL','NULL','Michael.Lee@k0QIl6mntspX.org','2452411',3304.87
+'AAAAAAAAJGJEBAAA','Mr.','Charles','Jackson','N',25,3,1945,'LIBERIA','NULL','Charles.Jackson@ziLxUxp0.edu','2452585',2983.56
+'AAAAAAAAJHBDBAAA','Mr.','Edgar','Hunt','Y',27,12,1932,'CAMEROON','NULL','Edgar.Hunt@Z.edu','2452284',2066.48
+'AAAAAAAAJJKMAAAA','Sir','Jimmy','Mcinnis','Y',25,1,1964,'SINGAPORE','NULL','Jimmy.Mcinnis@9QXGY2Fn18bZ.org','2452483',3542.97
+'AAAAAAAAJKACAAAA','Dr.','Arthur','Harris','N',9,3,1925,'SEYCHELLES','NULL','Arthur.Harris@Sj3QkYzfu.com','2452622',4920.90
+'AAAAAAAAKBOHAAAA','Mr.','NULL','Christian','NULL',NULL,10,NULL,'GUERNSEY','NULL','James.Christian@8r5T.com','2452522',2824.24
+'AAAAAAAAKGFEBAAA','Mr.','Federico','Bradshaw','N',11,5,1978,'MONTSERRAT','NULL','Federico.Bradshaw@uzthL55V.com','2452453',1633.59
+'AAAAAAAAKJDMAAAA','Miss','Andrew','Osborne','N',13,5,1954,'SUDAN','NULL','Andrew.Osborne@D.edu','2452495',2858.60
+'AAAAAAAAKKCMAAAA','Sir','Randall','Oliver','Y',15,6,1969,'GREECE','NULL','Randall.Oliver@lr7eEGq2qUT8sD2.edu','2452305',3917.80
+'AAAAAAAAKMJBAAAA','Sir','John','Kruse','N',25,9,1950,'NEW ZEALAND','NULL','John.Kruse@BgZ6x.edu','2452516',4032.72
+'AAAAAAAAKPPEBAAA','Mr.','Thomas','Kurtz','N',23,1,1953,'WALLIS AND FUTUNA','NULL','Thomas.Kurtz@Xd5JPu7M1.edu','2452571',2123.82
+'AAAAAAAALCECAAAA','Sir','Dennis','Silva','N',29,10,1972,'TAJIKISTAN','NULL','Dennis.Silva@cjk.com','2452425',1934.64
+'AAAAAAAALFAGBAAA','Sir','Alvin','Young','Y',24,10,1973,'MONACO','NULL','Alvin.Young@iit.org','2452518',4766.65
+'AAAAAAAALJMHBAAA','Dr.','Stephanie','Maloney','Y',19,12,1935,'MOZAMBIQUE','NULL','Stephanie.Maloney@xlkHMrqUSFcrHG.com','2452572',14950.15
+'AAAAAAAALKHEAAAA','Mr.','Neil','Delarosa','Y',6,1,1933,'SEYCHELLES','NULL','Neil.Delarosa@F26REbIyVtpkY.org','2452299',1575.84
+'AAAAAAAALPIOAAAA','Mr.','Kendrick','Swank','N',10,9,1983,'NIGERIA','NULL','Kendrick.Swank@KiHF1kfJUPMgrkxAtGtI.com','2452629',6283.76
+'AAAAAAAAMAHMAAAA','Sir','John','Nunez','Y',29,7,1963,'GIBRALTAR','NULL','John.Nunez@NXXNFCCxOFqDq6B.com','2452400',8665.51
+'AAAAAAAAMBIEAAAA','Dr.','Timothy','Bruce','Y',25,3,1938,'FIJI','NULL','Timothy.Bruce@gO.org','2452530',3006.00
+'AAAAAAAAMFGLAAAA','Dr.','Ellis','Stringer','N',30,3,1985,'VIET NAM','NULL','Ellis.Stringer@dLx2T0e7gAp.com','2452520',1646.55
+'AAAAAAAAMFNCAAAA','Miss','Sandra','Chavez','N',15,10,1936,'MAURITANIA','NULL','Sandra.Chavez@jNLsx1jyV7C.org','2452610',7062.75
+'AAAAAAAAMKHOAAAA','Miss','Kristie','Ross','N',6,1,1966,'BARBADOS','NULL','Kristie.Ross@p.edu','2452439',1842.95
+'AAAAAAAANDGCBAAA','Ms.','Sheila','Hull','N',23,11,1992,'EQUATORIAL GUINEA','NULL','Sheila.Hull@eDEnMk74Ukoo.com','2452416',3453.32
+'AAAAAAAANIAFBAAA','Dr.','Adrian','Holt','Y',27,3,1990,'SEYCHELLES','NULL','Adrian.Holt@eQCL.org','2452529',1233.10
+'AAAAAAAANJJGAAAA','Dr.','David','Smith','Y',20,7,1939,'GUINEA-BISSAU','NULL','David.Smith@q8KF56o7peqsXNf3I.com','2452371',3186.92
+'AAAAAAAANNBABAAA','Ms.','Cordelia','Carpenter','Y',19,8,1990,'PALAU','NULL','Cordelia.Carpenter@nsyrjxnu.com','2452487',5297.04
+'AAAAAAAANONGAAAA','Dr.','Joel','Rucker','N',15,2,1947,'BOLIVIA','NULL','Joel.Rucker@CaD.edu','2452289',1910.50
+'AAAAAAAAOCGFAAAA','Miss','Adam','Martin','Y',20,9,1970,'ARMENIA','NULL','Adam.Martin@0Sb2XqFA9.edu','2452626',3373.66
+'AAAAAAAAOFGEAAAA','Sir','John','Hopkins','N',9,4,1953,'MALTA','NULL','John.Hopkins@zsuCL.edu','2452508',5102.37
+'AAAAAAAAOFICAAAA','NULL','Ramona','Moore','N',NULL,NULL,1949,'NULL','NULL','Ramona.Moore@Zk.edu','NULL',1641.74
+'AAAAAAAAOFIHBAAA','Mr.','Francis','Wilhelm','Y',28,1,1985,'LEBANON','NULL','Francis.Wilhelm@R3.edu','2452291',11642.40
+'AAAAAAAAOLBHBAAA','Dr.','Nathaniel','Downs','Y',2,7,1944,'DOMINICA','NULL','Nathaniel.Downs@sdfiKzLFg7Et.org','2452514',7484.11
+'AAAAAAAAOLBHBAAA','Dr.','Nathaniel','Downs','Y',2,7,1944,'DOMINICA','NULL','Nathaniel.Downs@sdfiKzLFg7Et.org','2452514',8594.42
+'AAAAAAAAOMMCAAAA','Dr.','Roberta','Johnson','Y',15,3,1976,'ANTIGUA AND BARBUDA','NULL','Roberta.Johnson@uAvA75ZjM.com','2452581',1803.33
+'AAAAAAAAPEAIAAAA','Sir','William','Mason','Y',1,5,1977,'KYRGYZSTAN','NULL','William.Mason@RD4e6I1JfQkEL9.org','2452495',3683.03
+'AAAAAAAAPIHKAAAA','Dr.','Felix','Bailey','Y',8,7,1959,'TIMOR-LESTE','NULL','Felix.Bailey@0dKxiSYTnCjo5ZPe1.com','2452328',2856.00
+'AAAAAAAAPINOAAAA','Dr.','Michael','Johnson','Y',3,1,1976,'URUGUAY','NULL','Michael.Johnson@5l6aamKSUv6BV.org','2452531',8948.80
+'AAAAAAAAPJAGBAAA','Dr.','Michael','Stevens','N',28,2,1943,'GEORGIA','NULL','Michael.Stevens@c9ygsUdHa3J1XG.com','2452349',4491.28
+'AAAAAAAAPKCMAAAA','Dr.','Harold','Williams','N',13,9,1976,'QATAR','NULL','Harold.Williams@U44mtEQN0Qrm.org','2452409',1760.85
+'AAAAAAAAPNDDAAAA','Mr.','Tomas','Menendez','N',26,9,1934,'FINLAND','NULL','Tomas.Menendez@qibyGN3Ior.edu','2452573',1945.48
+---- TYPES
+STRING, STRING, STRING, STRING, STRING, INT, INT, INT, STRING, STRING, STRING, STRING, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q47.test b/testdata/workloads/tpcds/queries/tpcds-q47.test
new file mode 100644
index 0000000..b8481ff
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q47.test
@@ -0,0 +1,154 @@
+====
+---- QUERY: TPCDS-Q47
+with v1 as(
+ select i_category, i_brand,
+        s_store_name, s_company_name,
+        d_year, d_moy,
+        sum(ss_sales_price) sum_sales,
+        avg(sum(ss_sales_price)) over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     s_store_name, s_company_name
+           order by d_year, d_moy) rn
+ from item, store_sales, date_dim, store
+ where ss_item_sk = i_item_sk and
+       ss_sold_date_sk = d_date_sk and
+       ss_store_sk = s_store_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          s_store_name, s_company_name,
+          d_year, d_moy),
+ v2 as(
+ select v1.i_category
+        ,v1.d_year, v1.d_moy
+        ,v1.avg_monthly_sales
+        ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1.s_store_name = v1_lag.s_store_name and
+       v1.s_store_name = v1_lead.s_store_name and
+       v1.s_company_name = v1_lag.s_company_name and
+       v1.s_company_name = v1_lead.s_company_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+  select  *
+ from v2
+ where  d_year = 2000 and
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, 3
+ limit 100;
+---- RESULTS
+'Women',2000,6,5167.86,1702.17,3236.87,3399.13
+'Music',2000,4,5036.09,1860.99,2706.82,3303.43
+'Children',2000,3,5064.60,1912.86,3298.07,2757.48
+'Women',2000,3,4913.03,1779.75,2390.63,2257.04
+'Shoes',2000,7,5168.76,2069.14,2983.37,8035.14
+'Music',2000,3,5110.74,2022.81,2461.84,3067.59
+'Music',2000,5,4803.55,1743.74,2778.83,3066.97
+'Children',2000,7,5175.09,2116.90,3610.19,6870.27
+'Men',2000,4,4937.40,1880.07,3078.36,2756.00
+'Women',2000,3,4958.76,1958.78,2678.23,3027.08
+'Music',2000,3,5212.67,2215.63,3321.39,3357.05
+'Women',2000,6,4950.27,1956.92,2980.12,2196.47
+'Shoes',2000,6,4926.84,1942.09,2679.25,3503.44
+'Women',2000,5,4606.79,1639.46,2157.87,2583.06
+'Women',2000,1,4997.97,2033.94,4136.88,3386.81
+'Music',2000,3,5212.35,2256.79,2483.30,3203.46
+'Shoes',2000,3,5326.51,2380.52,3557.54,2381.65
+'Shoes',2000,4,5326.51,2381.65,2380.52,3185.08
+'Music',2000,6,5076.01,2166.79,2636.43,2696.26
+'Shoes',2000,5,5213.91,2308.61,3484.35,2631.62
+'Shoes',2000,7,4944.16,2043.50,2714.17,7158.49
+'Children',2000,6,5109.57,2231.87,2492.85,2716.69
+'Music',2000,7,5036.09,2162.98,2169.49,7442.03
+'Music',2000,6,5036.09,2169.49,3303.43,2162.98
+'Music',2000,3,4953.79,2087.58,2881.98,2396.23
+'Shoes',2000,3,4824.92,1981.97,2557.91,2178.72
+'Music',2000,4,5270.56,2429.81,2500.28,3121.27
+'Women',2000,4,4390.82,1562.28,2203.54,2814.98
+'Shoes',2000,2,4565.83,1740.10,3299.55,2905.54
+'Women',2000,4,4950.27,2125.36,3688.19,2980.12
+'Men',2000,7,4657.83,1836.07,2432.42,6304.84
+'Shoes',2000,6,5094.27,2285.80,2982.59,2944.67
+'Shoes',2000,5,5233.67,2458.45,3530.35,4587.71
+'Music',2000,3,5270.56,2500.28,3185.98,2429.81
+'Music',2000,3,5257.11,2491.28,2494.71,2963.86
+'Music',2000,2,5257.11,2494.71,2788.64,2491.28
+'Shoes',2000,4,5168.76,2407.05,3240.95,2483.46
+'Women',2000,3,4422.38,1666.68,2570.10,2097.53
+'Women',2000,7,4950.27,2196.47,1956.92,7068.51
+'Men',2000,5,5016.40,2267.73,2984.52,3051.68
+'Music',2000,4,5076.01,2328.83,3139.50,2636.43
+'Men',2000,4,4657.83,1925.05,2409.04,2999.06
+'Music',2000,2,4931.62,2199.12,3942.13,2335.05
+'Music',2000,2,5212.35,2483.30,4582.12,2256.79
+'Men',2000,3,4872.14,2143.87,3353.30,2688.70
+'Music',2000,7,5212.35,2487.88,2624.81,6788.15
+'Shoes',2000,7,4912.34,2194.71,3320.16,7019.04
+'Women',2000,4,4521.21,1803.95,2858.45,2451.32
+'Music',2000,7,5257.11,2540.19,3558.30,7261.59
+'Women',2000,2,5183.97,2471.71,3493.02,2920.36
+'Shoes',2000,3,4440.28,1737.33,1857.98,2951.57
+'Women',2000,4,5167.86,2468.50,3375.26,3236.87
+'Shoes',2000,2,4732.80,2035.56,3540.84,2823.10
+'Music',2000,2,4960.84,2267.45,3486.05,3329.26
+'Shoes',2000,2,5240.65,2549.66,3081.94,2798.03
+'Shoes',2000,5,5168.76,2483.46,2407.05,2983.37
+'Women',2000,5,4888.69,2204.11,3149.43,2688.11
+'Music',2000,7,5270.56,2587.41,3050.17,6970.69
+'Children',2000,7,4372.63,1694.02,2257.72,6000.24
+'Music',2000,4,4916.41,2240.65,3382.64,3182.24
+'Men',2000,6,4364.73,1689.53,2992.20,3120.76
+'Shoes',2000,4,5090.50,2415.99,3256.81,2925.27
+'Music',2000,4,4813.66,2143.57,2637.02,2902.70
+'Men',2000,5,4595.06,1926.36,2586.96,2922.71
+'Women',2000,4,4913.03,2257.04,1779.75,2643.30
+'Men',2000,4,4371.78,1718.21,2208.02,2721.25
+'Music',2000,2,5110.74,2461.84,4401.88,2022.81
+'Women',2000,3,5192.78,2544.73,3587.53,3197.85
+'Shoes',2000,4,4824.92,2178.72,1981.97,2884.49
+'Shoes',2000,3,5233.67,2588.01,3532.78,3530.35
+'Children',2000,5,4483.99,1838.66,2619.28,2374.55
+'Music',2000,7,5043.64,2403.81,3238.49,6835.65
+'Women',2000,7,4407.06,1776.42,2981.25,5710.21
+'Men',2000,6,4830.24,2199.80,2412.75,3006.00
+'Shoes',2000,6,5326.51,2701.75,3185.08,3285.18
+'Music',2000,2,4916.41,2298.33,3493.76,3382.64
+'Women',2000,5,4727.37,2110.50,2519.62,2267.53
+'Children',2000,5,5109.57,2492.85,2728.55,2231.87
+'Music',2000,4,5043.64,2430.73,2644.32,3481.08
+'Music',2000,6,4131.19,1519.23,2157.93,2655.31
+'Music',2000,2,5172.15,2563.98,2905.94,3777.26
+'Music',2000,4,4931.62,2327.08,2335.05,2882.04
+'Music',2000,5,4960.84,2360.20,2406.42,3555.36
+'Shoes',2000,4,4823.30,2222.80,3016.25,2462.91
+'Music',2000,3,4931.62,2335.05,2199.12,2327.08
+'Music',2000,6,5212.35,2624.81,2899.34,2487.88
+'Men',2000,3,5016.40,2432.36,3088.82,2984.52
+'Shoes',2000,2,4440.28,1857.98,3346.62,1737.33
+'Shoes',2000,6,5213.91,2631.62,2308.61,3081.21
+'Music',2000,6,5000.81,2419.42,3117.07,3086.00
+'Children',2000,3,4551.43,1973.72,2083.41,2621.49
+'Music',2000,7,5212.67,2638.50,3720.45,7180.32
+'Women',2000,2,5092.84,2525.38,3066.60,2819.83
+'Shoes',2000,2,5213.91,2649.38,3206.27,3351.30
+'Men',2000,1,4595.06,2032.46,3557.44,2372.72
+'Shoes',2000,7,5032.34,2469.75,2961.97,6219.49
+'Women',2000,6,4776.42,2216.70,2861.21,3620.51
+'Women',2000,7,4727.37,2168.76,2267.53,7819.76
+'Music',2000,4,4953.79,2396.23,2087.58,2921.95
+'Children',2000,5,5206.23,2649.03,2879.82,3447.77
+---- TYPES
+STRING, INT, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q48.test b/testdata/workloads/tpcds/queries/tpcds-q48.test
new file mode 100644
index 0000000..c08e538
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q48.test
@@ -0,0 +1,71 @@
+====
+---- QUERY: TPCDS-Q48
+select sum (ss_quantity)
+ from store_sales, store, customer_demographics, customer_address, date_dim
+ where s_store_sk = ss_store_sk
+ and  ss_sold_date_sk = d_date_sk and d_year = 1998
+ and
+ (
+  (
+   cd_demo_sk = ss_cdemo_sk
+   and
+   cd_marital_status = 'M'
+   and
+   cd_education_status = '4 yr Degree'
+   and
+   ss_sales_price between 100.00 and 150.00
+   )
+ or
+  (
+  cd_demo_sk = ss_cdemo_sk
+   and
+   cd_marital_status = 'M'
+   and
+   cd_education_status = '4 yr Degree'
+   and
+   ss_sales_price between 50.00 and 100.00
+  )
+ or
+ (
+  cd_demo_sk = ss_cdemo_sk
+  and
+   cd_marital_status = 'M'
+   and
+   cd_education_status = '4 yr Degree'
+   and
+   ss_sales_price between 150.00 and 200.00
+ )
+ )
+ and
+ (
+  (
+  ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('KY', 'GA', 'NM')
+  and ss_net_profit between 0 and 2000
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('MT', 'OR', 'IN')
+  and ss_net_profit between 150 and 3000
+  )
+ or
+  (ss_addr_sk = ca_address_sk
+  and
+  ca_country = 'United States'
+  and
+  ca_state in ('WI', 'MO', 'WV')
+  and ss_net_profit between 50 and 25000
+  )
+ )
+;
+---- RESULTS
+18742
+---- TYPES
+BIGINT
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q57.test b/testdata/workloads/tpcds/queries/tpcds-q57.test
new file mode 100644
index 0000000..c8d42ed
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q57.test
@@ -0,0 +1,151 @@
+====
+---- QUERY: TPCDS-Q57
+with v1 as(
+ select i_category, i_brand,
+        cc_name,
+        d_year, d_moy,
+        sum(cs_sales_price) sum_sales,
+        avg(sum(cs_sales_price)) over
+          (partition by i_category, i_brand,
+                     cc_name, d_year)
+          avg_monthly_sales,
+        rank() over
+          (partition by i_category, i_brand,
+                     cc_name
+           order by d_year, d_moy) rn
+ from item, catalog_sales, date_dim, call_center
+ where cs_item_sk = i_item_sk and
+       cs_sold_date_sk = d_date_sk and
+       cc_call_center_sk= cs_call_center_sk and
+       (
+         d_year = 2000 or
+         ( d_year = 2000-1 and d_moy =12) or
+         ( d_year = 2000+1 and d_moy =1)
+       )
+ group by i_category, i_brand,
+          cc_name , d_year, d_moy),
+ v2 as(
+ select v1.i_category, v1.i_brand
+        ,v1.d_year, v1.d_moy
+        ,v1.avg_monthly_sales
+        ,v1.sum_sales, v1_lag.sum_sales psum, v1_lead.sum_sales nsum
+ from v1, v1 v1_lag, v1 v1_lead
+ where v1.i_category = v1_lag.i_category and
+       v1.i_category = v1_lead.i_category and
+       v1.i_brand = v1_lag.i_brand and
+       v1.i_brand = v1_lead.i_brand and
+       v1. cc_name = v1_lag. cc_name and
+       v1. cc_name = v1_lead. cc_name and
+       v1.rn = v1_lag.rn + 1 and
+       v1.rn = v1_lead.rn - 1)
+  select  *
+ from v2
+ where  d_year = 2000 and
+        avg_monthly_sales > 0 and
+        case when avg_monthly_sales > 0 then abs(sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+ order by sum_sales - avg_monthly_sales, 3
+ limit 100;
+---- RESULTS
+'Music','exportischolar #2',2000,2,7201.09,2787.29,3877.25,4135.16
+'Music','edu packscholar #2',2000,3,7565.59,3181.66,3343.32,4565.20
+'Women','importoamalg #2',2000,4,7052.31,2758.84,5323.46,3397.55
+'Music','edu packscholar #2',2000,2,7565.59,3343.32,4935.00,3181.66
+'Children','edu packexporti #2',2000,2,6345.20,2210.46,3973.84,3221.55
+'Shoes','amalgedu pack #2',2000,2,7209.57,3085.07,3565.30,4091.47
+'Music','exportischolar #2',2000,4,7201.09,3133.86,4135.16,4593.34
+'Music','edu packscholar #2',2000,6,7565.59,3524.34,5698.93,3734.51
+'Women','amalgamalg #2',2000,4,6500.18,2467.19,2614.22,4090.61
+'Music','importoscholar #2',2000,5,7209.97,3179.64,4922.03,3375.16
+'Shoes','importoedu pack #2',2000,4,7799.84,3779.41,5531.49,4100.04
+'Shoes','edu packedu pack #2',2000,3,6570.57,2582.43,3361.18,4247.21
+'Women','amalgamalg #2',2000,5,6974.84,2986.83,4369.56,4425.63
+'Music','exportischolar #2',2000,4,7189.19,3209.91,4070.55,4736.81
+'Shoes','exportiedu pack #2',2000,7,6282.87,2323.03,3618.89,8561.10
+'Shoes','importoedu pack #2',2000,5,7451.17,3497.68,4029.18,3736.68
+'Shoes','amalgedu pack #2',2000,5,7209.57,3288.79,3876.29,5356.13
+'Women','amalgamalg #2',2000,3,6500.18,2614.22,4120.11,2467.19
+'Music','importoscholar #2',2000,6,7173.54,3305.49,4124.85,5517.87
+'Music','importoscholar #2',2000,5,7333.86,3476.24,4186.38,4310.57
+'Women','edu packamalg #2',2000,4,7077.02,3221.39,4578.37,3950.64
+'Music','importoscholar #2',2000,6,7209.97,3375.16,3179.64,4200.45
+'Music','edu packscholar #2',2000,7,7565.59,3734.51,3524.34,10911.39
+'Music','edu packscholar #2',2000,4,7393.93,3566.55,4683.15,4192.86
+'Children','importoexporti #2',2000,4,6736.02,2938.80,4408.60,3459.57
+'Music','edu packscholar #2',2000,6,7393.93,3605.67,4192.86,4584.42
+'Children','importoexporti #2',2000,5,7750.52,3964.46,5069.64,5289.29
+'Children','importoexporti #2',2000,2,7750.52,3970.89,4680.36,5186.78
+'Music','importoscholar #2',2000,1,7173.54,3425.31,7158.98,3653.76
+'Shoes','importoedu pack #2',2000,6,7003.12,3271.77,4315.53,3844.47
+'Shoes','amalgedu pack #2',2000,7,7547.71,3817.97,4325.72,10447.92
+'Shoes','importoedu pack #2',2000,6,7451.17,3736.68,3497.68,4623.33
+'Women','importoamalg #2',2000,7,7052.31,3341.72,4560.01,10274.76
+'Children','importoexporti #2',2000,6,6736.02,3027.47,3459.57,3790.37
+'Shoes','amalgedu pack #2',2000,1,7547.71,3844.74,5202.59,4396.86
+'Shoes','importoedu pack #2',2000,5,7799.84,4100.04,3779.41,4264.74
+'Shoes','amalgedu pack #2',2000,5,7547.71,3869.87,5245.43,4325.72
+'Music','amalgscholar #2',2000,1,6166.31,2490.48,7622.41,2511.57
+'Children','importoexporti #2',2000,3,7239.69,3569.22,3791.71,3662.60
+'Music','exportischolar #2',2000,2,7180.31,3517.49,4135.97,5162.68
+'Women','importoamalg #2',2000,5,7052.31,3397.55,2758.84,4560.01
+'Music','amalgscholar #2',2000,2,6166.31,2511.57,2490.48,4536.16
+'Men','importoimporto #2',2000,4,6884.73,3238.62,4209.58,3839.59
+'Shoes','amalgedu pack #2',2000,1,7209.57,3565.30,8102.54,3085.07
+'Children','importoexporti #2',2000,4,7239.69,3662.60,3569.22,5525.78
+'Women','edu packamalg #2',2000,6,7358.00,3787.73,5088.83,5803.55
+'Shoes','exportiedu pack #2',2000,1,6443.39,2873.80,7194.15,3855.27
+'Music','exportischolar #2',2000,6,7180.31,3612.29,5020.81,4207.36
+'Children','edu packexporti #2',2000,3,6527.41,2961.33,4500.62,3803.47
+'Music','importoscholar #2',2000,1,7209.97,3647.90,6746.00,4352.48
+'Music','exportischolar #2',2000,6,7189.19,3629.32,4736.81,3981.89
+'Women','edu packamalg #2',2000,5,7291.36,3736.99,4426.42,4092.15
+'Shoes','edu packedu pack #2',2000,5,6600.88,3047.98,3510.15,3138.61
+'Shoes','importoedu pack #2',2000,6,7799.84,4264.74,4100.04,4268.37
+'Shoes','importoedu pack #2',2000,7,7799.84,4268.37,4264.74,10510.36
+'Shoes','importoedu pack #2',2000,2,7451.17,3929.11,4573.75,4039.83
+'Music','importoscholar #2',2000,2,7173.54,3653.76,3425.31,3947.95
+'Women','edu packamalg #2',2000,2,7077.02,3564.97,5471.15,4578.37
+'Men','importoimporto #2',2000,4,6918.52,3411.02,3709.06,4760.77
+'Women','importoamalg #2',2000,7,7334.01,3849.94,4082.40,9171.27
+'Men','edu packimporto #2',2000,4,5950.79,2470.54,3971.86,3035.58
+'Shoes','edu packedu pack #2',2000,6,6600.88,3138.61,3047.98,4619.22
+'Shoes','amalgedu pack #2',2000,7,7209.57,3749.49,5356.13,9885.01
+'Shoes','edu packedu pack #2',2000,1,6600.88,3142.19,6834.22,3525.41
+'Children','importoexporti #2',2000,2,7239.69,3791.71,5862.92,3569.22
+'Women','importoamalg #2',2000,4,7334.01,3908.07,4004.54,4347.22
+'Shoes','importoedu pack #2',2000,4,7451.17,4029.18,4039.83,3497.68
+'Shoes','edu packedu pack #2',2000,7,6573.14,3153.26,4172.69,9299.79
+'Men','importoimporto #2',2000,6,6918.52,3499.04,4760.77,3845.54
+'Children','edu packexporti #2',2000,4,6507.45,3094.38,3556.30,4301.81
+'Women','edu packamalg #2',2000,2,7291.36,3879.78,4986.31,4063.80
+'Shoes','importoedu pack #2',2000,3,7451.17,4039.83,3929.11,4029.18
+'Women','exportiamalg #2',2000,1,6440.52,3035.08,5330.23,3319.38
+'Women','importoamalg #2',2000,6,7114.52,3731.58,4087.83,3968.11
+'Shoes','edu packedu pack #2',2000,4,6573.14,3198.44,3446.93,3735.20
+'Shoes','exportiedu pack #2',2000,6,6323.38,2950.42,3321.16,3547.45
+'Men','exportiimporto #2',2000,2,6155.52,2805.25,3204.31,4158.65
+'Women','exportiamalg #2',2000,6,6567.90,3219.25,4349.74,4169.37
+'Children','exportiexporti #2',2000,7,5789.60,2442.48,3626.57,9336.54
+'Music','edu packscholar #2',2000,2,7393.93,4054.26,4200.58,4683.15
+'Men','amalgimporto #2',2000,2,6097.26,2760.55,3906.28,3409.11
+'Women','importoamalg #2',2000,1,7114.52,3780.48,7156.48,4503.18
+'Shoes','amalgedu pack #2',2000,4,7209.57,3876.29,4091.47,3288.79
+'Men','importoimporto #2',2000,7,6884.73,3551.59,4528.39,8535.51
+'Women','importoamalg #2',2000,3,7334.01,4004.54,4128.85,3908.07
+'Music','exportischolar #2',2000,1,7201.09,3877.25,7562.58,2787.29
+'Shoes','exportiedu pack #2',2000,4,6282.87,2962.87,4050.37,3568.63
+'Children','amalgexporti #2',2000,2,6735.91,3434.61,4322.80,3825.99
+'Men','edu packimporto #2',2000,4,6081.16,2781.12,4744.62,3539.63
+'Women','amalgamalg #2',2000,7,6974.84,3676.16,4425.63,8099.82
+'Children','edu packexporti #2',2000,5,6527.41,3239.36,3803.47,3355.26
+'Shoes','amalgedu pack #2',2000,5,6576.22,3290.55,4295.49,3497.97
+'Music','importoscholar #2',2000,4,7173.54,3896.97,3947.95,4124.85
+'Children','importoexporti #2',2000,5,6736.02,3459.57,2938.80,3027.47
+'Men','amalgimporto #2',2000,4,5789.47,2519.48,2917.13,3100.93
+'Children','amalgexporti #2',2000,7,5990.68,2722.94,3144.36,8294.07
+'Women','exportiamalg #2',2000,4,6440.52,3178.75,4514.55,4645.14
+'Women','amalgamalg #2',2000,1,6974.84,3719.07,7297.36,4378.50
+'Children','importoexporti #2',2000,7,7750.52,4494.79,5289.29,8873.39
+'Men','importoimporto #2',2000,2,6884.73,3629.86,4031.03,4209.58
+---- TYPES
+STRING, STRING, INT, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q58.test b/testdata/workloads/tpcds/queries/tpcds-q58.test
new file mode 100644
index 0000000..e36b55f
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q58.test
@@ -0,0 +1,68 @@
+====
+---- QUERY: TPCDS-Q58
+with ss_items as
+ (select i_item_id item_id
+        ,sum(ss_ext_sales_price) ss_item_rev
+ from store_sales
+     ,item
+     ,date_dim
+ where ss_item_sk = i_item_sk
+   and d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+   and ss_sold_date_sk   = d_date_sk
+ group by i_item_id),
+ cs_items as
+ (select i_item_id item_id
+        ,sum(cs_ext_sales_price) cs_item_rev
+  from catalog_sales
+      ,item
+      ,date_dim
+ where cs_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq = (select d_week_seq
+                                      from date_dim
+                                      where d_date = '1998-02-19'))
+  and  cs_sold_date_sk = d_date_sk
+ group by i_item_id),
+ ws_items as
+ (select i_item_id item_id
+        ,sum(ws_ext_sales_price) ws_item_rev
+  from web_sales
+      ,item
+      ,date_dim
+ where ws_item_sk = i_item_sk
+  and  d_date in (select d_date
+                  from date_dim
+                  where d_week_seq =(select d_week_seq
+                                     from date_dim
+                                     where d_date = '1998-02-19'))
+  and ws_sold_date_sk   = d_date_sk
+ group by i_item_id)
+  select  ss_items.item_id
+       ,ss_item_rev
+       ,ss_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ss_dev
+       ,cs_item_rev
+       ,cs_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 cs_dev
+       ,ws_item_rev
+       ,ws_item_rev/(ss_item_rev+cs_item_rev+ws_item_rev)/3 * 100 ws_dev
+       ,(ss_item_rev+cs_item_rev+ws_item_rev)/3 average
+ from ss_items,cs_items,ws_items
+ where ss_items.item_id=cs_items.item_id
+   and ss_items.item_id=ws_items.item_id
+   and ss_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+   and ss_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+   and cs_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+   and cs_item_rev between 0.9 * ws_item_rev and 1.1 * ws_item_rev
+   and ws_item_rev between 0.9 * ss_item_rev and 1.1 * ss_item_rev
+   and ws_item_rev between 0.9 * cs_item_rev and 1.1 * cs_item_rev
+ order by item_id
+         ,ss_item_rev
+ limit 100;
+---- RESULTS
+---- TYPES
+STRING, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q59.test b/testdata/workloads/tpcds/queries/tpcds-q59.test
new file mode 100644
index 0000000..9288c18
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q59.test
@@ -0,0 +1,147 @@
+====
+---- QUERY: TPCDS-Q59
+with wss as
+ (select d_week_seq,
+        ss_store_sk,
+        sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales,
+        sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales,
+        sum(case when (d_day_name='Tuesday') then ss_sales_price else  null end) tue_sales,
+        sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales,
+        sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales,
+        sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales,
+        sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales
+ from store_sales,date_dim
+ where d_date_sk = ss_sold_date_sk
+ group by d_week_seq,ss_store_sk
+ )
+  select  s_store_name1,s_store_id1,d_week_seq1
+       ,sun_sales1/sun_sales2,mon_sales1/mon_sales2
+       ,tue_sales1/tue_sales1,wed_sales1/wed_sales2,thu_sales1/thu_sales2
+       ,fri_sales1/fri_sales2,sat_sales1/sat_sales2
+ from
+ (select s_store_name s_store_name1,wss.d_week_seq d_week_seq1
+        ,s_store_id s_store_id1,sun_sales sun_sales1
+        ,mon_sales mon_sales1,tue_sales tue_sales1
+        ,wed_sales wed_sales1,thu_sales thu_sales1
+        ,fri_sales fri_sales1,sat_sales sat_sales1
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and
+        d_month_seq between 1185 and 1185 + 11) y,
+ (select s_store_name s_store_name2,wss.d_week_seq d_week_seq2
+        ,s_store_id s_store_id2,sun_sales sun_sales2
+        ,mon_sales mon_sales2,tue_sales tue_sales2
+        ,wed_sales wed_sales2,thu_sales thu_sales2
+        ,fri_sales fri_sales2,sat_sales sat_sales2
+  from wss,store,date_dim d
+  where d.d_week_seq = wss.d_week_seq and
+        ss_store_sk = s_store_sk and
+        d_month_seq between 1185+ 12 and 1185 + 23) x
+ where s_store_id1=s_store_id2
+   and d_week_seq1=d_week_seq2-52
+ order by s_store_name1,s_store_id1,d_week_seq1
+limit 100;
+---- RESULTS
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5153,0.72,1.28,1.00,0.91,1.08,1.43,1.05
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5154,1.10,0.89,1.00,1.15,1.10,0.89,1.69
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+'able','AAAAAAAACAAAAAAA',5155,1.09,1.19,1.00,0.89,1.13,0.72,1.03
+---- TYPES
+STRING, STRING, INT, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q63.test b/testdata/workloads/tpcds/queries/tpcds-q63.test
new file mode 100644
index 0000000..1fc417a
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q63.test
@@ -0,0 +1,76 @@
+====
+---- QUERY: TPCDS-Q63
+select  *
+from (select i_manager_id
+             ,sum(ss_sales_price) sum_sales
+             ,avg(sum(ss_sales_price)) over (partition by i_manager_id) avg_monthly_sales
+      from item
+          ,store_sales
+          ,date_dim
+          ,store
+      where ss_item_sk = i_item_sk
+        and ss_sold_date_sk = d_date_sk
+        and ss_store_sk = s_store_sk
+        and d_month_seq in (1212,1212+1,1212+2,1212+3,1212+4,1212+5,1212+6,1212+7,1212+8,1212+9,1212+10,1212+11)
+        and ((    i_category in ('Books','Children','Electronics')
+              and i_class in ('personal','portable','refernece','self-help')
+              and i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7',
+                                  'exportiunivamalg #9','scholaramalgamalg #9'))
+           or(    i_category in ('Women','Music','Men')
+              and i_class in ('accessories','classical','fragrances','pants')
+              and i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1',
+                                 'importoamalg #1')))
+group by i_manager_id, d_moy) tmp1
+where case when avg_monthly_sales > 0 then abs (sum_sales - avg_monthly_sales) / avg_monthly_sales else null end > 0.1
+order by i_manager_id
+        ,avg_monthly_sales
+        ,sum_sales
+limit 100;
+---- RESULTS
+10,58.29,219.28
+10,86.13,219.28
+10,99.79,219.28
+10,103.41,219.28
+10,123.23,219.28
+10,145.67,219.28
+10,180.92,219.28
+10,246.39,219.28
+10,439.43,219.28
+10,457.26,219.28
+10,480.87,219.28
+13,11.61,170.40
+13,11.82,170.40
+13,71.49,170.40
+13,75.64,170.40
+13,84.31,170.40
+13,113.69,170.40
+13,253.69,170.40
+13,291.33,170.40
+13,304.78,170.40
+13,461.69,170.40
+29,4.08,171.75
+29,18.16,171.75
+29,44.87,171.75
+29,71.17,171.75
+29,102.90,171.75
+29,138.61,171.75
+29,208.57,171.75
+29,247.06,171.75
+29,248.03,171.75
+29,372.54,171.75
+29,433.35,171.75
+34,5.95,175.05
+34,13.06,175.05
+34,32.31,175.05
+34,76.42,175.05
+34,82.49,175.05
+34,112.40,175.05
+34,126.59,175.05
+34,210.32,175.05
+34,219.27,175.05
+34,258.47,175.05
+34,409.37,175.05
+34,554.01,175.05
+---- TYPES
+INT, DECIMAL, DECIMAL
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-q83.test b/testdata/workloads/tpcds/queries/tpcds-q83.test
new file mode 100644
index 0000000..91f5691
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q83.test
@@ -0,0 +1,74 @@
+====
+---- QUERY: TPCDS-Q83
+with sr_items as
+ (select i_item_id item_id,
+        sum(sr_return_quantity) sr_item_qty
+ from store_returns,
+      item,
+      date_dim
+ where sr_item_sk = i_item_sk
+ and   d_date    in
+        (select d_date
+        from date_dim
+        where d_week_seq in
+                (select d_week_seq
+                from date_dim
+          where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   sr_returned_date_sk   = d_date_sk
+ group by i_item_id),
+ cr_items as
+ (select i_item_id item_id,
+        sum(cr_return_quantity) cr_item_qty
+ from catalog_returns,
+      item,
+      date_dim
+ where cr_item_sk = i_item_sk
+ and   d_date    in
+       (select d_date
+       from date_dim
+       where d_week_seq in
+               (select d_week_seq
+               from date_dim
+         where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and  cr_returned_date_sk   = d_date_sk
+ group by i_item_id),
+ wr_items as
+ (select i_item_id item_id,
+        sum(wr_return_quantity) wr_item_qty
+ from web_returns,
+      item,
+      date_dim
+ where wr_item_sk = i_item_sk
+ and   d_date    in
+        (select d_date
+        from date_dim
+        where d_week_seq in
+                (select d_week_seq
+                from date_dim
+                where d_date in ('1998-01-02','1998-10-15','1998-11-10')))
+ and   wr_returned_date_sk   = d_date_sk
+ group by i_item_id)
+  select  sr_items.item_id
+       ,sr_item_qty
+       ,sr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 sr_dev
+       ,cr_item_qty
+       ,cr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 cr_dev
+       ,wr_item_qty
+       ,wr_item_qty/(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 * 100 wr_dev
+       ,(sr_item_qty+cr_item_qty+wr_item_qty)/3.0 average
+ from sr_items
+     ,cr_items
+     ,wr_items
+ where sr_items.item_id=cr_items.item_id
+   and sr_items.item_id=wr_items.item_id
+ order by sr_items.item_id
+         ,sr_item_qty
+ limit 100;
+---- RESULTS
+'AAAAAAAAAHKAAAAA',24,11.26760563380282,39,18.30985915492958,8,3.755868544600939,23.66666666666667
+'AAAAAAAAIFNDAAAA',4,1.801801801801802,10,4.504504504504505,60,27.02702702702703,24.66666666666667
+'AAAAAAAAKJADAAAA',17,8.095238095238095,51,24.28571428571428,2,0.9523809523809523,23.33333333333333
+'AAAAAAAAOAIBAAAA',40,13.60544217687075,32,10.8843537414966,26,8.843537414965986,32.66666666666666
+---- TYPES
+STRING, BIGINT, DOUBLE, BIGINT, DOUBLE, BIGINT, DOUBLE, DOUBLE
+====
diff --git a/testdata/workloads/tpcds/queries/tpcds-q85.test b/testdata/workloads/tpcds/queries/tpcds-q85.test
new file mode 100644
index 0000000..d387f8a
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q85.test
@@ -0,0 +1,90 @@
+====
+---- QUERY: TPCDS-Q85
+select  substr(r_reason_desc,1,20)
+       ,avg(ws_quantity)
+       ,avg(wr_refunded_cash)
+       ,avg(wr_fee)
+ from web_sales, web_returns, web_page, customer_demographics cd1,
+      customer_demographics cd2, customer_address, date_dim, reason
+ where ws_web_page_sk = wp_web_page_sk
+   and ws_item_sk = wr_item_sk
+   and ws_order_number = wr_order_number
+   and ws_sold_date_sk = d_date_sk and d_year = 1998
+   and cd1.cd_demo_sk = wr_refunded_cdemo_sk
+   and cd2.cd_demo_sk = wr_returning_cdemo_sk
+   and ca_address_sk = wr_refunded_addr_sk
+   and r_reason_sk = wr_reason_sk
+   and
+   (
+    (
+     cd1.cd_marital_status = 'M'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = '4 yr Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 100.00 and 150.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'D'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Primary'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 50.00 and 100.00
+    )
+   or
+    (
+     cd1.cd_marital_status = 'U'
+     and
+     cd1.cd_marital_status = cd2.cd_marital_status
+     and
+     cd1.cd_education_status = 'Advanced Degree'
+     and
+     cd1.cd_education_status = cd2.cd_education_status
+     and
+     ws_sales_price between 150.00 and 200.00
+    )
+   )
+   and
+   (
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('KY', 'GA', 'NM')
+     and ws_net_profit between 100 and 200
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('MT', 'OR', 'IN')
+     and ws_net_profit between 150 and 300
+    )
+    or
+    (
+     ca_country = 'United States'
+     and
+     ca_state in ('WI', 'MO', 'WV')
+     and ws_net_profit between 50 and 250
+    )
+   )
+group by r_reason_desc
+order by substr(r_reason_desc,1,20)
+        ,avg(ws_quantity)
+        ,avg(wr_refunded_cash)
+        ,avg(wr_fee)
+limit 100;
+---- RESULTS
+'Did not like the mak',31,50.21,95.17
+'Found a better price',7,11.86,27.89
+'Not the product that',5,107.38,69.70
+---- TYPES
+STRING, DOUBLE, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/testdata/workloads/tpcds/queries/tpcds-q89.test b/testdata/workloads/tpcds/queries/tpcds-q89.test
new file mode 100644
index 0000000..ff580b8
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-q89.test
@@ -0,0 +1,131 @@
+====
+---- QUERY: TPCDS-Q89
+select  *
+from(
+select i_category, i_class, i_brand,
+       s_store_name, s_company_name,
+       d_moy,
+       sum(ss_sales_price) sum_sales,
+       avg(sum(ss_sales_price)) over
+         (partition by i_category, i_brand, s_store_name, s_company_name)
+         avg_monthly_sales
+from item, store_sales, date_dim, store
+where ss_item_sk = i_item_sk and
+      ss_sold_date_sk = d_date_sk and
+      ss_store_sk = s_store_sk and
+      d_year in (2000) and
+        ((i_category in ('Home','Books','Electronics') and
+          i_class in ('wallpaper','parenting','musical')
+         )
+      or (i_category in ('Shoes','Jewelry','Men') and
+          i_class in ('womens','birdal','pants')
+        ))
+group by i_category, i_class, i_brand,
+         s_store_name, s_company_name, d_moy) tmp1
+where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1
+order by sum_sales - avg_monthly_sales, s_store_name
+limit 100;
+---- RESULTS
+'Shoes','womens','amalgedu pack #2','eing','Unknown',7,2043.50,4944.16
+'Shoes','womens','amalgedu pack #2','able','Unknown',2,2035.56,4732.80
+'Men','pants','exportiimporto #2','bar','Unknown',6,1689.53,4364.73
+'Men','pants','exportiimporto #2','eing','Unknown',4,1783.84,4269.09
+'Shoes','womens','amalgedu pack #2','able','Unknown',5,2250.32,4732.80
+'Shoes','womens','amalgedu pack #2','ation','Unknown',6,2271.73,4711.60
+'Shoes','womens','amalgedu pack #2','eing','Unknown',4,2516.16,4944.16
+'Shoes','womens','amalgedu pack #2','ese','Unknown',2,2212.92,4631.16
+'Shoes','womens','amalgedu pack #2','ation','Unknown',7,2293.91,4711.60
+'Men','pants','exportiimporto #2','able','Unknown',4,2114.49,4471.48
+'Shoes','womens','amalgedu pack #2','ese','Unknown',4,2335.75,4631.16
+'Shoes','womens','amalgedu pack #2','able','Unknown',4,2471.45,4732.80
+'Shoes','womens','amalgedu pack #2','eing','Unknown',6,2714.17,4944.16
+'Shoes','womens','amalgedu pack #2','ought','Unknown',7,2385.30,4582.19
+'Men','pants','exportiimporto #2','ese','Unknown',2,2002.22,4191.21
+'Men','pants','exportiimporto #2','ought','Unknown',1,2013.92,4160.93
+'Men','pants','exportiimporto #2','ese','Unknown',6,2045.85,4191.21
+'Shoes','womens','amalgedu pack #2','eing','Unknown',3,2818.29,4944.16
+'Shoes','womens','amalgedu pack #2','ought','Unknown',2,2484.68,4582.19
+'Shoes','womens','amalgedu pack #2','ation','Unknown',3,2617.58,4711.60
+'Shoes','womens','amalgedu pack #2','ought','Unknown',4,2489.01,4582.19
+'Shoes','womens','amalgedu pack #2','bar','Unknown',7,2473.32,4563.12
+'Men','pants','exportiimporto #2','bar','Unknown',3,2284.67,4364.73
+'Shoes','womens','amalgedu pack #2','able','Unknown',6,2661.93,4732.80
+'Shoes','womens','amalgedu pack #2','bar','Unknown',2,2518.10,4563.12
+'Men','pants','exportiimporto #2','eing','Unknown',2,2228.12,4269.09
+'Men','pants','exportiimporto #2','ought','Unknown',5,2123.43,4160.93
+'Men','pants','exportiimporto #2','able','Unknown',3,2442.59,4471.48
+'Shoes','womens','amalgedu pack #2','eing','Unknown',2,2943.53,4944.16
+'Shoes','womens','amalgedu pack #2','ation','Unknown',4,2730.48,4711.60
+'Men','pants','exportiimporto #2','ought','Unknown',4,2191.09,4160.93
+'Men','pants','exportiimporto #2','eing','Unknown',5,2303.47,4269.09
+'Shoes','womens','amalgedu pack #2','ought','Unknown',3,2636.91,4582.19
+'Shoes','womens','amalgedu pack #2','ation','Unknown',5,2780.52,4711.60
+'Men','pants','exportiimporto #2','ation','Unknown',4,2225.47,4154.93
+'Shoes','womens','amalgedu pack #2','ese','Unknown',6,2703.10,4631.16
+'Men','pants','exportiimporto #2','ation','Unknown',2,2241.65,4154.93
+'Shoes','womens','amalgedu pack #2','able','Unknown',3,2823.10,4732.80
+'Shoes','womens','amalgedu pack #1','ation','Unknown',2,752.99,2646.46
+'Men','pants','exportiimporto #2','able','Unknown',5,2601.64,4471.48
+'Shoes','womens','amalgedu pack #2','able','Unknown',7,2864.95,4732.80
+'Shoes','womens','amalgedu pack #2','ese','Unknown',3,2768.79,4631.16
+'Men','pants','exportiimporto #2','ese','Unknown',5,2401.63,4191.21
+'Shoes','womens','amalgedu pack #2','bar','Unknown',6,2799.28,4563.12
+'Men','pants','exportiimporto #2','ation','Unknown',5,2421.85,4154.93
+'Shoes','womens','amalgedu pack #2','ought','Unknown',6,2853.29,4582.19
+'Men','pants','exportiimporto #2','ought','Unknown',2,2432.48,4160.93
+'Men','pants','exportiimporto #2','able','Unknown',6,2760.92,4471.48
+'Men','pants','exportiimporto #2','eing','Unknown',6,2559.62,4269.09
+'Men','pants','exportiimporto #2','ation','Unknown',3,2453.59,4154.93
+'Shoes','womens','amalgedu pack #2','bar','Unknown',4,2950.74,4563.12
+'Men','pants','exportiimporto #2','ation','Unknown',1,2566.01,4154.93
+'Shoes','womens','amalgedu pack #2','eing','Unknown',1,3373.13,4944.16
+'Shoes','womens','amalgedu pack #1','bar','Unknown',2,746.66,2311.21
+'Men','pants','exportiimporto #2','ese','Unknown',7,2631.49,4191.21
+'Shoes','womens','amalgedu pack #2','ation','Unknown',2,3154.14,4711.60
+'Men','pants','exportiimporto #2','able','Unknown',1,2918.84,4471.48
+'Shoes','womens','amalgedu pack #2','ese','Unknown',1,3089.25,4631.16
+'Shoes','womens','amalgedu pack #2','bar','Unknown',3,3050.52,4563.12
+'Men','pants','exportiimporto #1','eing','Unknown',4,583.03,2095.40
+'Shoes','womens','amalgedu pack #1','eing','Unknown',6,991.32,2497.71
+'Men','pants','exportiimporto #2','ation','Unknown',6,2649.32,4154.93
+'Men','pants','exportiimporto #2','able','Unknown',7,2973.59,4471.48
+'Shoes','womens','amalgedu pack #1','bar','Unknown',3,828.20,2311.21
+'Men','pants','exportiimporto #2','ought','Unknown',7,2701.30,4160.93
+'Shoes','womens','amalgedu pack #1','eing','Unknown',4,1049.89,2497.71
+'Shoes','womens','amalgedu pack #2','bar','Unknown',5,3126.30,4563.12
+'Shoes','womens','amalgedu pack #1','ation','Unknown',7,1230.33,2646.46
+'Men','pants','exportiimporto #2','ation','Unknown',7,2739.09,4154.93
+'Shoes','womens','amalgedu pack #2','ought','Unknown',5,3167.99,4582.19
+'Men','pants','exportiimporto #2','bar','Unknown',4,2977.49,4364.73
+'Shoes','womens','amalgedu pack #2','eing','Unknown',5,3557.58,4944.16
+'Men','pants','exportiimporto #2','bar','Unknown',5,2992.20,4364.73
+'Men','pants','exportiimporto #2','able','Unknown',2,3099.75,4471.48
+'Shoes','womens','amalgedu pack #2','ought','Unknown',1,3214.28,4582.19
+'Men','pants','exportiimporto #2','ought','Unknown',3,2795.06,4160.93
+'Shoes','womens','amalgedu pack #1','able','Unknown',6,1155.46,2507.54
+'Shoes','womens','amalgedu pack #2','bar','Unknown',1,3217.46,4563.12
+'Shoes','womens','amalgedu pack #1','ought','Unknown',7,1142.24,2459.56
+'Shoes','womens','amalgedu pack #1','able','Unknown',2,1203.83,2507.54
+'Men','pants','exportiimporto #1','eing','Unknown',7,797.62,2095.40
+'Men','pants','exportiimporto #2','eing','Unknown',1,2980.60,4269.09
+'Shoes','womens','amalgedu pack #1','ation','Unknown',4,1359.31,2646.46
+'Shoes','womens','amalgedu pack #1','ought','Unknown',2,1174.72,2459.56
+'Men','pants','exportiimporto #1','bar','Unknown',5,671.59,1948.48
+'Men','pants','exportiimporto #2','bar','Unknown',2,3099.19,4364.73
+'Shoes','womens','amalgedu pack #1','able','Unknown',3,1248.26,2507.54
+'Men','pants','exportiimporto #2','bar','Unknown',7,3120.76,4364.73
+'Shoes','womens','amalgedu pack #1','ation','Unknown',6,1407.42,2646.46
+'Shoes','womens','amalgedu pack #1','ese','Unknown',4,1285.46,2520.39
+'Men','pants','exportiimporto #2','ese','Unknown',1,2962.47,4191.21
+'Shoes','womens','amalgedu pack #1','ought','Unknown',4,1248.31,2459.56
+'Shoes','womens','amalgedu pack #2','ation','Unknown',1,3507.39,4711.60
+'Shoes','womens','amalgedu pack #2','able','Unknown',1,3540.84,4732.80
+'Men','pants','exportiimporto #1','ese','Unknown',4,720.85,1911.92
+'Men','pants','exportiimporto #1','able','Unknown',6,741.13,1929.06
+'Men','pants','exportiimporto #2','ought','Unknown',6,2973.45,4160.93
+'Shoes','womens','amalgedu pack #1','able','Unknown',7,1326.81,2507.54
+'Shoes','womens','amalgedu pack #1','ought','Unknown',3,1290.11,2459.56
+'Men','pants','exportiimporto #2','bar','Unknown',1,3201.04,4364.73
+---- TYPES
+STRING, STRING, STRING, STRING, STRING, INT, DECIMAL, DECIMAL
+====
\ No newline at end of file
diff --git a/tests/query_test/test_tpcds_queries.py b/tests/query_test/test_tpcds_queries.py
index 2d26309..512fbf5 100644
--- a/tests/query_test/test_tpcds_queries.py
+++ b/tests/query_test/test_tpcds_queries.py
@@ -109,9 +109,15 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q25(self, vector):
     self.run_test_case(self.get_workload() + '-q25', vector)
 
+  def test_tpcds_q26(self, vector):
+    self.run_test_case(self.get_workload() + '-q26', vector)
+
   def test_tpcds_q29(self, vector):
     self.run_test_case(self.get_workload() + '-q29', vector)
 
+  def test_tpcds_q30(self, vector):
+    self.run_test_case(self.get_workload() + '-q30', vector)
+
   def test_tpcds_q32(self, vector):
     self.run_test_case(self.get_workload() + '-q32', vector)
 
@@ -145,6 +151,12 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q46(self, vector):
     self.run_test_case(self.get_workload() + '-q46', vector)
 
+  def test_tpcds_q47(self, vector):
+    self.run_test_case(self.get_workload() + '-q47', vector)
+
+  def test_tpcds_q48(self, vector):
+    self.run_test_case(self.get_workload() + '-q48', vector)
+
   def test_tpcds_q50(self, vector):
     self.run_test_case(self.get_workload() + '-q50', vector)
 
@@ -169,6 +181,15 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q56(self, vector):
     self.run_test_case(self.get_workload() + '-q56', vector)
 
+  def test_tpcds_q57(self, vector):
+    self.run_test_case(self.get_workload() + '-q57', vector)
+
+  def test_tpcds_q58(self, vector):
+    self.run_test_case(self.get_workload() + '-q58', vector)
+
+  def test_tpcds_q59(self, vector):
+    self.run_test_case(self.get_workload() + '-q59', vector)
+
   def test_tpcds_q60(self, vector):
     self.run_test_case(self.get_workload() + '-q60', vector)
 
@@ -178,6 +199,9 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q62(self, vector):
     self.run_test_case(self.get_workload() + '-q62', vector)
 
+  def test_tpcds_q63(self, vector):
+    self.run_test_case(self.get_workload() + '-q63', vector)
+
   def test_tpcds_q64(self, vector):
     self.run_test_case(self.get_workload() + '-q64', vector)
 
@@ -232,15 +256,24 @@ class TestTpcdsQuery(ImpalaTestSuite):
   def test_tpcds_q82(self, vector):
     self.run_test_case(self.get_workload() + '-q82', vector)
 
+  def test_tpcds_q83(self, vector):
+    self.run_test_case(self.get_workload() + '-q83', vector)
+
   def test_tpcds_q84(self, vector):
     self.run_test_case(self.get_workload() + '-q84', vector)
 
+  def test_tpcds_q85(self, vector):
+    self.run_test_case(self.get_workload() + '-q85', vector)
+
   def test_tpcds_q86a(self, vector):
     self.run_test_case(self.get_workload() + '-q86a', vector)
 
   def test_tpcds_q88(self, vector):
     self.run_test_case(self.get_workload() + '-q88', vector)
 
+  def test_tpcds_q89(self, vector):
+    self.run_test_case(self.get_workload() + '-q89', vector)
+
   def test_tpcds_q91(self, vector):
     self.run_test_case(self.get_workload() + '-q91', vector)
 
@@ -348,9 +381,18 @@ class TestTpcdsDecimalV2Query(ImpalaTestSuite):
   def test_tpcds_q25(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q25', vector)
 
+  def test_tpcds_q26(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q26', vector)
+
   def test_tpcds_q29(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q29', vector)
 
+  def test_tpcds_q30(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q30', vector)
+
+  def test_tpcds_q31(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q31', vector)
+
   def test_tpcds_q32(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q32', vector)
 
@@ -384,6 +426,12 @@ class TestTpcdsDecimalV2Query(ImpalaTestSuite):
   def test_tpcds_q46(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q46', vector)
 
+  def test_tpcds_q47(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q47', vector)
+
+  def test_tpcds_q48(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q48', vector)
+
   def test_tpcds_q50(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q50', vector)
 
@@ -408,6 +456,15 @@ class TestTpcdsDecimalV2Query(ImpalaTestSuite):
   def test_tpcds_q56(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q56', vector)
 
+  def test_tpcds_q57(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q57', vector)
+
+  def test_tpcds_q58(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q58', vector)
+
+  def test_tpcds_q59(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q59', vector)
+
   def test_tpcds_q60(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q60', vector)
 
@@ -417,6 +474,9 @@ class TestTpcdsDecimalV2Query(ImpalaTestSuite):
   def test_tpcds_q62(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q62', vector)
 
+  def test_tpcds_q63(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q63', vector)
+
   def test_tpcds_q64(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q64', vector)
 
@@ -471,15 +531,24 @@ class TestTpcdsDecimalV2Query(ImpalaTestSuite):
   def test_tpcds_q82(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q82', vector)
 
+  def test_tpcds_q83(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q83', vector)
+
   def test_tpcds_q84(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q84', vector)
 
+  def test_tpcds_q85(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q85', vector)
+
   def test_tpcds_q86a(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q86a', vector)
 
   def test_tpcds_q88(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q88', vector)
 
+  def test_tpcds_q89(self, vector):
+    self.run_test_case(self.get_workload() + '-decimal_v2-q89', vector)
+
   def test_tpcds_q91(self, vector):
     self.run_test_case(self.get_workload() + '-decimal_v2-q91', vector)
 
diff --git a/tests/util/parse_util.py b/tests/util/parse_util.py
index efe56b7..4aeb00b 100644
--- a/tests/util/parse_util.py
+++ b/tests/util/parse_util.py
@@ -22,7 +22,7 @@ from datetime import datetime
 # changed, and the stress test loses the ability to run the full set of queries. Set
 # these constants and assert that when a workload is used, all the queries we expect to
 # use are there.
-EXPECTED_TPCDS_QUERIES_COUNT = 72
+EXPECTED_TPCDS_QUERIES_COUNT = 84
 EXPECTED_TPCH_NESTED_QUERIES_COUNT = 22
 EXPECTED_TPCH_QUERIES_COUNT = 22
 # Add the number of stress test specific queries, i.e. in files like '*-stress-*.test'


[impala] 02/03: IMPALA-9569: Fix progress bar and live_summary to show info of the retried query

Posted by st...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1fbca6d43bc3fb08b9e04a4eb8e1c1dcf7c67712
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Thu Jun 18 21:45:26 2020 +0800

    IMPALA-9569: Fix progress bar and live_summary to show info of the retried query
    
    Impala-shell periodically calls GetExecSummary() when the query is
    queuing or running. If the query is being retried, GetExecSummary()
    should return the TExecSummary of the retried query. So the progress bar
    and live_summary can reflect the most recent state.
    
    This patch also modifies get_summary() to return retry information in
    error_logs of TExecSummary. Impala-shell and other clients can print the
    info right after the query starts being retried. Modified impala-shell
    to print the retried query link when the retried query is running.
    
    Example output when the retried query is running:
    Query: select count(*) from functional.alltypes where bool_col = sleep(60)
    Query submitted at: 2020-06-18 22:08:49 (Coordinator: http://quanlong-OptiPlex-BJ:25000)
    Query progress can be monitored at: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=9444fe7f0df0da28:29134b0800000000
    Failed due to unreachable impalad(s): quanlong-OptiPlex-BJ:22001
    
    Retrying query using query id: 5748d9a3ccc28ba8:a75e2fab00000000
    Retried query link: http://quanlong-OptiPlex-BJ:25000/query_plan?query_id=5748d9a3ccc28ba8:a75e2fab00000000
    [###############################                               ] 50%
    
    Tests:
    - Manually verify the progress bar and live_summary work when the query
      is being retried.
    - Add tests in test_query_retries.py to validate the get_summary()
      results.
    
    Change-Id: I8f96919f00e0b64d589efd15b6b5ec82fb725d56
    Reviewed-on: http://gerrit.cloudera.org:8080/16096
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/service/impala-server.cc            | 35 ++++++++++--
 be/src/service/impala-server.h             |  8 +++
 shell/impala_shell.py                      | 14 ++++-
 tests/common/impala_connection.py          |  5 +-
 tests/custom_cluster/test_query_retries.py | 86 +++++++++++++++++++++++-------
 5 files changed, 120 insertions(+), 28 deletions(-)

diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index ac5dfd6..f34bc41 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -713,8 +713,11 @@ Status ImpalaServer::GetExecSummary(const TUniqueId& query_id, const string& use
     TExecSummary* result) {
   // Search for the query id in the active query map.
   {
+    // QueryHandle of the current query.
     QueryHandle query_handle;
-    Status status = GetQueryHandle(query_id, &query_handle,
+    // QueryHandle or the original query if the query is retried.
+    QueryHandle original_query_handle;
+    Status status = GetAllQueryHandles(query_id, &query_handle, &original_query_handle,
         /*return_unregistered=*/ true);
     if (status.ok()) {
       lock_guard<mutex> l(*query_handle->lock());
@@ -733,7 +736,6 @@ Status ImpalaServer::GetExecSummary(const TUniqueId& query_id, const string& use
             }
           }
         }
-        return Status::OK();
       } else if (query_handle->GetCoordinator() != nullptr) {
         query_handle->GetCoordinator()->GetTExecSummary(result);
         TExecProgress progress;
@@ -743,11 +745,18 @@ Status ImpalaServer::GetExecSummary(const TUniqueId& query_id, const string& use
             query_handle->GetCoordinator()->progress().total());
         // TODO: does this not need to be synchronized?
         result->__set_progress(progress);
-        return Status::OK();
       } else {
         *result = TExecSummary();
-        return Status::OK();
       }
+      if (query_handle->IsRetriedQuery()) {
+        // Don't need to acquire lock on original_query_handle since the query is
+        // finished. There are no concurrent updates on its status.
+        result->error_logs.push_back(original_query_handle->query_status().GetDetail());
+        result->error_logs.push_back(Substitute("Retrying query using query id: $0",
+            PrintId(query_handle->query_id())));
+        result->__isset.error_logs = true;
+      }
+      return Status::OK();
     }
   }
 
@@ -1338,6 +1347,24 @@ Status ImpalaServer::GetQueryHandle(
   return Status::OK();
 }
 
+Status ImpalaServer::GetAllQueryHandles(const TUniqueId& query_id,
+    QueryHandle* active_query_handle, QueryHandle* original_query_handle,
+    bool return_unregistered) {
+  DCHECK(active_query_handle != nullptr);
+  DCHECK(original_query_handle != nullptr);
+  shared_ptr<QueryDriver> query_driver = GetQueryDriver(query_id, return_unregistered);
+  if (UNLIKELY(query_driver == nullptr)) {
+    Status err = Status::Expected(TErrorCode::INVALID_QUERY_HANDLE, PrintId(query_id));
+    VLOG(1) << err.GetDetail();
+    return err;
+  }
+  active_query_handle->SetHandle(query_driver,
+      query_driver->GetActiveClientRequestState());
+  original_query_handle->SetHandle(query_driver,
+      query_driver->GetClientRequestState(query_id));
+  return Status::OK();
+}
+
 Status ImpalaServer::CancelInternal(const TUniqueId& query_id) {
   VLOG_QUERY << "Cancel(): query_id=" << PrintId(query_id);
   QueryHandle query_handle;
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index 5fc66e8..1945534 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -938,6 +938,14 @@ class ImpalaServer : public ImpalaServiceIf,
   Status GetQueryHandle(const TUniqueId& query_id, QueryHandle* query_handle,
       bool return_unregistered = false);
 
+  /// Returns both the active QueryHandle and the original QueryHandle for this query id.
+  /// In scenarios that require both QueryHandles, calling 'GetActiveQueryHandle' and
+  /// 'GetQueryHandle' one by one may have race conditions with QueryDriver deletion
+  /// causing QueryDriver not found in the second call. Use this method in such cases.
+  /// See 'GetQueryDriver' for a description of the 'return_unregistered' parameter.
+  Status GetAllQueryHandles(const TUniqueId& query_id, QueryHandle* active_query_handle,
+      QueryHandle* original_query_handle, bool return_unregistered = false);
+
   /// Returns the QueryDriver for the given query_id, or nullptr if not found. If
   /// 'return_unregistered' is true, queries that have started unregistration
   /// may be returned. Otherwise queries that have started unregistration will
diff --git a/shell/impala_shell.py b/shell/impala_shell.py
index 87436f5..e0d8026 100755
--- a/shell/impala_shell.py
+++ b/shell/impala_shell.py
@@ -1088,6 +1088,17 @@ class ImpalaShell(cmd.Cmd, object):
         self.last_summary = time.time()
         return
 
+      data = ""
+      if summary.error_logs:
+        for error_line in summary.error_logs:
+          data += error_line + "\n"
+          query_id_search = re.search("Retrying query using query id: (.*)",
+                                      error_line)
+          if query_id_search and len(query_id_search.groups()) == 1:
+            retried_query_id = query_id_search.group(1)
+            data += "Retried query link: %s\n"\
+                    % self.imp_client.get_query_link(retried_query_id)
+
       if summary.progress:
         progress = summary.progress
 
@@ -1096,7 +1107,6 @@ class ImpalaShell(cmd.Cmd, object):
           self.last_summary = time.time()
           return
 
-        data = ""
         if self.live_progress and progress.total_scan_ranges > 0:
           val = ((summary.progress.num_completed_scan_ranges * 100) /
                  summary.progress.total_scan_ranges)
@@ -1110,7 +1120,7 @@ class ImpalaShell(cmd.Cmd, object):
           formatter = PrettyOutputFormatter(table)
           data += formatter.format(output) + "\n"
 
-        self.progress_stream.write(data)
+      self.progress_stream.write(data)
       self.last_summary = time.time()
 
   def _default_summary_table(self):
diff --git a/tests/common/impala_connection.py b/tests/common/impala_connection.py
index 1e24abc..0fee7d9 100644
--- a/tests/common/impala_connection.py
+++ b/tests/common/impala_connection.py
@@ -385,8 +385,9 @@ class ImpylaHS2Connection(ImpalaConnection):
 
   def get_exec_summary(self, operation_handle):
     LOG.info("-- getting exec summary operation: {0}".format(operation_handle))
-    raise NotImplementedError(
-        "Not yet implemented for HS2 - summary returned is thrift, not string.")
+    cursor = operation_handle.get_handle()
+    # summary returned is thrift, not string.
+    return cursor.get_summary()
 
   def get_runtime_profile(self, operation_handle, profile_format):
     LOG.info("-- getting runtime profile operation: {0}".format(operation_handle))
diff --git a/tests/custom_cluster/test_query_retries.py b/tests/custom_cluster/test_query_retries.py
index c21f57f..e17b807 100644
--- a/tests/custom_cluster/test_query_retries.py
+++ b/tests/custom_cluster/test_query_retries.py
@@ -77,10 +77,14 @@ class TestQueryRetries(CustomClusterTestSuite):
     assert len(results.data) == 1
     assert int(results.data[0]) == 3650
 
+    # Validate the live exec summary.
+    retried_query_id = self.__get_retried_query_id_from_summary(handle)
+    assert retried_query_id is not None
+
     # Validate the state of the runtime profiles.
     retried_runtime_profile = self.client.get_runtime_profile(handle)
-    retried_query_id =\
-        self.__validate_runtime_profiles(retried_runtime_profile, handle.get_handle().id)
+    self.__validate_runtime_profiles(
+        retried_runtime_profile, handle.get_handle().id, retried_query_id)
 
     # Validate the state of the client log.
     self.__validate_client_log(handle, retried_query_id)
@@ -115,6 +119,10 @@ class TestQueryRetries(CustomClusterTestSuite):
     assert len(results.data) == 1
     assert self._shuffle_heavy_query_results in results.data[0]
 
+    # Validate the live exec summary.
+    retried_query_id = self.__get_retried_query_id_from_summary(handle)
+    assert retried_query_id is not None
+
     # The runtime profile of the retried query.
     retried_runtime_profile = self.client.get_runtime_profile(handle)
 
@@ -123,8 +131,8 @@ class TestQueryRetries(CustomClusterTestSuite):
     self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
 
     # Validate the state of the runtime profiles.
-    retried_query_id = self.__validate_runtime_profiles(
-        retried_runtime_profile, handle.get_handle().id)
+    self.__validate_runtime_profiles(
+        retried_runtime_profile, handle.get_handle().id, retried_query_id)
 
     # Validate the state of the client log.
     self.__validate_client_log(handle, retried_query_id)
@@ -183,8 +191,11 @@ class TestQueryRetries(CustomClusterTestSuite):
       retried_runtime_profile = self.client.get_runtime_profile(handle)
       self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
 
-      retried_query_id = self.__validate_runtime_profiles(
-          retried_runtime_profile, handle.get_handle().id)
+      retried_query_id = self.__get_retried_query_id_from_summary(handle)
+      assert retried_query_id is not None
+
+      self.__validate_runtime_profiles(
+          retried_runtime_profile, handle.get_handle().id, retried_query_id)
 
       self.__validate_client_log(handle, retried_query_id)
 
@@ -227,9 +238,13 @@ class TestQueryRetries(CustomClusterTestSuite):
     # the runtime profile.
     self.__assert_executors_blacklisted(killed_impalad, retried_runtime_profile)
 
+    # Validate the live exec summary.
+    retried_query_id = self.__get_retried_query_id_from_summary(handle)
+    assert retried_query_id is not None
+
     # Validate the state of the runtime profiles.
-    retried_query_id = self.__validate_runtime_profiles(
-      retried_runtime_profile, handle.get_handle().id)
+    self.__validate_runtime_profiles(
+        retried_runtime_profile, handle.get_handle().id, retried_query_id)
 
     # Validate the state of the client log.
     self.__validate_client_log(handle, retried_query_id)
@@ -267,11 +282,14 @@ class TestQueryRetries(CustomClusterTestSuite):
     # Wait until the query fails.
     self.wait_for_state(handle, self.client.QUERY_STATES['EXCEPTION'], 60)
 
+    # Validate the live exec summary.
+    retried_query_id = self.__get_retried_query_id_from_summary(handle)
+    assert retried_query_id is not None
+
     # The runtime profile and client log of the retried query, need to be retrieved
     # before fetching results, since the failed fetch attempt will close the
     # query handle.
     retried_runtime_profile = self.client.get_runtime_profile(handle)
-    retried_query_id = self.__get_query_id_from_profile(retried_runtime_profile)
     self.__validate_client_log(handle, retried_query_id)
 
     # Assert that the query failed, since a query can only be retried once.
@@ -334,9 +352,14 @@ class TestQueryRetries(CustomClusterTestSuite):
         query_options={'retry_failed_queries': 'true'})
     self.wait_for_state(handle, self.client.QUERY_STATES['FINISHED'], 60)
 
+    # Validate the live exec summary.
+    retried_query_id = self.__get_retried_query_id_from_summary(handle)
+    assert retried_query_id is not None
+
     # Validate that the query was retried.
-    retried_query_id = self.__validate_runtime_profiles_from_service(
-        impalad_service, handle)
+    profile_retried_query_id = \
+        self.__validate_runtime_profiles_from_service(impalad_service, handle)
+    assert profile_retried_query_id == retried_query_id
     self.__validate_client_log(handle, retried_query_id)
 
     # Cancel the query.
@@ -369,12 +392,17 @@ class TestQueryRetries(CustomClusterTestSuite):
         query_options={'retry_failed_queries': 'true', 'query_timeout_s': '1'})
     self.wait_for_state(handle, self.client.QUERY_STATES['EXCEPTION'], 60)
 
+    # Validate the live exec summary.
+    retried_query_id = self.__get_retried_query_id_from_summary(handle)
+    assert retried_query_id is not None
+
     # Wait for the query timeout to expire the query handle.
     time.sleep(5)
 
     # Validate that the query was retried.
-    retried_query_id = self.__validate_runtime_profiles_from_service(
-        impalad_service, handle)
+    profile_retried_query_id = \
+        self.__validate_runtime_profiles_from_service(impalad_service, handle)
+    assert profile_retried_query_id == retried_query_id
     self.__validate_client_log(handle, retried_query_id)
 
     # Assert than attempt to fetch from the query handle fails with a query expired
@@ -439,11 +467,16 @@ class TestQueryRetries(CustomClusterTestSuite):
     assert len(results.data) == 1
     assert int(results.data[0]) == 6001215
 
+    # Validate the live exec summary.
+    retried_query_id = \
+        self.__get_retried_query_id_from_summary(handle, use_hs2_client=True)
+    assert retried_query_id is not None
+
     # Validate the state of the runtime profiles.
     retried_runtime_profile = self.hs2_client.get_runtime_profile(handle,
         TRuntimeProfileFormat.STRING)
-    retried_query_id = self.__validate_runtime_profiles(
-        retried_runtime_profile, self.hs2_client.get_query_id(handle))
+    self.__validate_runtime_profiles(
+        retried_runtime_profile, self.hs2_client.get_query_id(handle), retried_query_id)
     self.__validate_client_log(handle, retried_query_id, use_hs2_client=True)
     self.impalad_test_service.wait_for_metric_value(
         'impala-server.resultset-cache.total-num-rows', 1, timeout=60)
@@ -455,7 +488,8 @@ class TestQueryRetries(CustomClusterTestSuite):
     original_profile = impalad_service.read_query_profile_page(handle.get_handle().id)
     retried_query_id = self.__get_retried_query_id_from_profile(original_profile)
     retried_profile = impalad_service.read_query_profile_page(retried_query_id)
-    self.__validate_runtime_profiles(retried_profile, handle.get_handle().id)
+    self.__validate_runtime_profiles(
+        retried_profile, handle.get_handle().id, retried_query_id)
     return retried_query_id
 
   def __get_retried_query_id_from_profile(self, profile):
@@ -515,7 +549,8 @@ class TestQueryRetries(CustomClusterTestSuite):
             original_id_pattern, retried_runtime_profile)
     assert original_id_search.group(1) == original_query_id
 
-  def __validate_runtime_profiles(self, retried_runtime_profile, original_query_id):
+  def __validate_runtime_profiles(self, retried_runtime_profile, original_query_id,
+                                  retried_query_id):
     """"Validate the runtime profiles of both the original and retried queries. The
     'retried_runtime_profile' refers to the runtime profile of the retried query (the
     most recent attempt of the query, which should have succeeded). The
@@ -523,8 +558,8 @@ class TestQueryRetries(CustomClusterTestSuite):
     original attempt of the query submitted by the user, which failed and had to be
     retried)."""
 
-    # Extract the retried query id from the retried runtime profile.
-    retried_query_id = self.__get_query_id_from_profile(retried_runtime_profile)
+    # Check the retried query id in the retried runtime profile.
+    assert retried_query_id == self.__get_query_id_from_profile(retried_runtime_profile)
 
     # Assert that the query id of the original query is in the runtime profile of the
     # retried query.
@@ -540,7 +575,6 @@ class TestQueryRetries(CustomClusterTestSuite):
     # Assert that the query options from the original and retried queries are the same.
     assert self.__get_query_options(original_runtime_profile) == \
         self.__get_query_options(retried_runtime_profile)
-    return retried_query_id
 
   def __get_query_options(self, profile):
     """Returns the query options from the given profile."""
@@ -599,3 +633,15 @@ class TestQueryRetries(CustomClusterTestSuite):
     assert query_id_search,\
       "Invalid client log, has no retried query id. Log=%s" % client_log
     assert query_id_search.group(1) == retried_query_id
+
+  def __get_retried_query_id_from_summary(self, handle, use_hs2_client=False):
+    if use_hs2_client:
+      summary = self.hs2_client.get_exec_summary(handle)
+    else:
+      summary = self.client.get_exec_summary(handle)
+    if summary.error_logs:
+      for log in summary.error_logs:
+        query_id_search = re.search("Retrying query using query id: (.*)", log)
+        if query_id_search:
+          return query_id_search.group(1)
+    return None