You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jo...@apache.org on 2022/09/23 15:47:54 UTC

[impala] branch master updated (3f382b7eb -> 296e94411)

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

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


    from 3f382b7eb IMPALA-11583: Use Iceberg API to update stats
     new 81e36d458 IMPALA-10660: Impala shell prints DOUBLEs with less precision in HS2 than beeswax
     new 296e94411 IMPALA-11599/IMPALA-11605: GCC 10: Fix gdb and change to flatbuffers 1.9

The 2 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:
 bin/impala-config.sh                   |  4 +-
 fe/pom.xml                             |  2 +-
 shell/impala_client.py                 | 84 ++++++++++++++++++----------------
 shell/impala_shell.py                  | 45 ++++++++++++++++--
 shell/impala_shell_config_defaults.py  |  1 +
 shell/make_shell_tarball.sh            |  1 +
 shell/option_parser.py                 |  7 ++-
 shell/packaging/make_python_package.sh |  1 +
 shell/value_converter.py               | 51 +++++++++++++++++++++
 tests/shell/test_shell_commandline.py  | 78 +++++++++++++++++++++++++++++++
 10 files changed, 227 insertions(+), 47 deletions(-)
 create mode 100644 shell/value_converter.py


[impala] 01/02: IMPALA-10660: Impala shell prints DOUBLEs with less precision in HS2 than beeswax

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

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

commit 81e36d45844e41e0edf5f51c03b3abdae1094cbd
Author: Peter Rozsa <pr...@cloudera.com>
AuthorDate: Fri Sep 9 18:30:50 2022 +0200

    IMPALA-10660: Impala shell prints DOUBLEs with less precision in HS2 than beeswax
    
    This change adds a shell option called "hs2_fp_format"
    which manipulates the print format of floating-point values in HS2.
    It lets the user to specify a Python-based format specification
    expression (https://docs.python.org/2.7/library/string.html#formatspec)
    which will get parsed and applied to floating-point
    column values. The default value is None, in this case the
    formatting is the same as the state before this change.
    This option does not support the Beeswax protocol, because Beeswax
    converts all of the column values to strings in its response.
    
    Tests: command line tests for various formatting options and
           for invalid formatting option
    
    Change-Id: I424339266be66437941be8bafaa83fa0f2dfbd4e
    Reviewed-on: http://gerrit.cloudera.org:8080/18990
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 shell/impala_client.py                 | 84 ++++++++++++++++++----------------
 shell/impala_shell.py                  | 45 ++++++++++++++++--
 shell/impala_shell_config_defaults.py  |  1 +
 shell/make_shell_tarball.sh            |  1 +
 shell/option_parser.py                 |  7 ++-
 shell/packaging/make_python_package.sh |  1 +
 shell/value_converter.py               | 51 +++++++++++++++++++++
 tests/shell/test_shell_commandline.py  | 78 +++++++++++++++++++++++++++++++
 8 files changed, 224 insertions(+), 44 deletions(-)

diff --git a/shell/impala_client.py b/shell/impala_client.py
index 77bab9efe..da7e63581 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -51,35 +51,32 @@ from thrift.Thrift import TApplicationException, TException
 from shell_exceptions import (RPCException, QueryStateException, DisconnectedException,
     QueryCancelledByShellException, MissingThriftMethodException, HttpError)
 
+from value_converter import HS2ValueConverter
 
-# Helpers to extract and convert HS2's representation of values to the display version.
+# Getters to extract HS2's representation of values to the display version.
 # An entry must be added to this map for each supported type. HS2's TColumn has many
-# different typed field, each of which has a 'values' and a 'nulls' field. The first
-# element of each tuple is a "getter" function that extracts the appropriate member from
-# TColumn for the given TTypeId. The second element is a "stringifier" function that
-# converts a single value to its display representation. If the value is already a
-# string and does not need conversion for display, the stringifier can be None.
-HS2_VALUE_CONVERTERS = {
-    TTypeId.BOOLEAN_TYPE: (operator.attrgetter('boolVal'),
-     lambda b: 'true' if b else 'false'),
-    TTypeId.TINYINT_TYPE: (operator.attrgetter('byteVal'), str),
-    TTypeId.SMALLINT_TYPE: (operator.attrgetter('i16Val'), str),
-    TTypeId.INT_TYPE: (operator.attrgetter('i32Val'), str),
-    TTypeId.BIGINT_TYPE: (operator.attrgetter('i64Val'), str),
-    TTypeId.TIMESTAMP_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.FLOAT_TYPE: (operator.attrgetter('doubleVal'), str),
-    TTypeId.DOUBLE_TYPE: (operator.attrgetter('doubleVal'), str),
-    TTypeId.STRING_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.DECIMAL_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.BINARY_TYPE: (operator.attrgetter('binaryVal'), str),
-    TTypeId.VARCHAR_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.CHAR_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.MAP_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.ARRAY_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.STRUCT_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.UNION_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.NULL_TYPE: (operator.attrgetter('stringVal'), None),
-    TTypeId.DATE_TYPE: (operator.attrgetter('stringVal'), None)
+# different typed field, each of which has a 'values' and a 'nulls' field. These getters
+# extract the appropriate member from TColumn for the given TTypeId.
+HS2_VALUE_GETTERS = {
+    TTypeId.BOOLEAN_TYPE: operator.attrgetter('boolVal'),
+    TTypeId.TINYINT_TYPE: operator.attrgetter('byteVal'),
+    TTypeId.SMALLINT_TYPE: operator.attrgetter('i16Val'),
+    TTypeId.INT_TYPE: operator.attrgetter('i32Val'),
+    TTypeId.BIGINT_TYPE: operator.attrgetter('i64Val'),
+    TTypeId.TIMESTAMP_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.FLOAT_TYPE: operator.attrgetter('doubleVal'),
+    TTypeId.DOUBLE_TYPE: operator.attrgetter('doubleVal'),
+    TTypeId.STRING_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.DECIMAL_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.BINARY_TYPE: operator.attrgetter('binaryVal'),
+    TTypeId.VARCHAR_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.CHAR_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.MAP_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.ARRAY_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.STRUCT_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.UNION_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.NULL_TYPE: operator.attrgetter('stringVal'),
+    TTypeId.DATE_TYPE: operator.attrgetter('stringVal')
 }
 
 
@@ -133,7 +130,7 @@ class ImpalaClient(object):
                kerberos_service_name="impala", use_ssl=False, ca_cert=None, user=None,
                ldap_password=None, use_ldap=False, client_connect_timeout_ms=60000,
                verbose=True, use_http_base_transport=False, http_path=None,
-               http_cookie_names=None, http_socket_timeout_s=None):
+               http_cookie_names=None, http_socket_timeout_s=None, value_converter=None):
     self.connected = False
     self.impalad_host = impalad[0]
     self.impalad_port = int(impalad[1])
@@ -162,6 +159,7 @@ class ImpalaClient(object):
     # 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
+    self.value_converter = value_converter
 
   def connect(self):
     """Creates a connection to an Impalad instance. Returns a tuple with the impala
@@ -660,6 +658,11 @@ class ImpalaHS2Client(ImpalaClient):
     # Minimum sleep interval between retry attempts.
     self.min_sleep_interval = 1
 
+    # In case of direct instantiation of the client where the converter is
+    # not set, there should be a default value converter assigned
+    if self.value_converter is None:
+      self.value_converter = HS2ValueConverter()
+
   def _get_thrift_client(self, protocol):
     return ImpalaHiveServer2Service.Client(protocol)
 
@@ -826,7 +829,9 @@ class ImpalaHS2Client(ImpalaClient):
     assert query_handle.hasResultSet
     prim_types = [column.typeDesc.types[0].primitiveEntry.type
                   for column in query_handle.schema.columns]
-    col_value_converters = [HS2_VALUE_CONVERTERS[prim_type]
+    column_value_getters = [HS2_VALUE_GETTERS[prim_type]
+                        for prim_type in prim_types]
+    column_value_converters = [self.value_converter.get_converter(prim_type)
                         for prim_type in prim_types]
     while True:
       req = TFetchResultsReq(query_handle, TFetchOrientation.FETCH_NEXT,
@@ -842,26 +847,27 @@ class ImpalaHS2Client(ImpalaClient):
       # Transpose the columns into a row-based format for more convenient processing
       # for the display code. This is somewhat inefficient, but performance is comparable
       # to the old Beeswax code.
-      yield self._transpose(col_value_converters, resp.results.columns)
-      if not self._hasMoreRows(resp, col_value_converters):
+      yield self._transpose(column_value_getters, column_value_converters,
+                            resp.results.columns)
+      if not self._hasMoreRows(resp, column_value_getters):
         return
 
-  def _hasMoreRows(self, resp, col_value_converters):
+  def _hasMoreRows(self, resp, column_value_getters):
     return resp.hasMoreRows
 
-  def _transpose(self, col_value_converters, columns):
+  def _transpose(self, column_value_getters, column_value_converters, columns):
     """Transpose the columns from a TFetchResultsResp into the row format returned
     by fetch() with all the values converted into their string representations for
-    display. Uses the getters and stringifiers provided in col_value_converters[i]
-    for column i."""
-    tcols = [col_value_converters[i][0](col) for i, col in enumerate(columns)]
+    display. Uses the getters and convertes provided in column_value_getters[i] and
+    column_value_converters[i] for column i."""
+    tcols = [column_value_getters[i](col) for i, col in enumerate(columns)]
     num_rows = len(tcols[0].values)
     # Preallocate rows for efficiency.
     rows = [[None] * len(tcols) for i in xrange(num_rows)]
     for col_idx, tcol in enumerate(tcols):
       is_null = bitarray(endian='little')
       is_null.frombytes(tcol.nulls)
-      stringifier = col_value_converters[col_idx][1]
+      stringifier = column_value_converters[col_idx]
       # Skip stringification if not needed. This makes large extracts of tpch.orders
       # ~8% faster according to benchmarks.
       if stringifier is None:
@@ -1140,8 +1146,8 @@ class StrictHS2Client(ImpalaHS2Client):
   def _populate_query_options(self):
     return
 
-  def _hasMoreRows(self, resp, col_value_converters):
-    tcol = col_value_converters[0][0](resp.results.columns[0])
+  def _hasMoreRows(self, resp, column_value_getters):
+    tcol = column_value_getters[0](resp.results.columns[0])
     return len(tcol.values)
 
 
diff --git a/shell/impala_shell.py b/shell/impala_shell.py
index 6e79e0a81..b25e47955 100755
--- a/shell/impala_shell.py
+++ b/shell/impala_shell.py
@@ -49,6 +49,8 @@ from subprocess import call
 from shell_exceptions import (RPCException, DisconnectedException, QueryStateException,
     QueryCancelledByShellException, MissingThriftMethodException)
 
+from value_converter import HS2ValueConverter
+
 
 VERSION_FORMAT = "Impala Shell v%(version)s (%(git_hash)s) built on %(build_date)s"
 VERSION_STRING = "impala shell build version not available"
@@ -589,6 +591,13 @@ class ImpalaShell(cmd.Cmd, object):
 
   def _new_impala_client(self):
     protocol = options.protocol.lower()
+
+    value_converter = None
+    if protocol == 'hs2' or protocol == 'hs2-http':
+      value_converter = HS2ValueConverter()
+      if options.hs2_fp_format:
+        value_converter.override_floating_point_converter(options.hs2_fp_format)
+
     if options.strict_hs2_protocol:
       assert protocol == 'hs2' or protocol == 'hs2-http'
       if protocol == 'hs2':
@@ -597,21 +606,22 @@ class ImpalaShell(cmd.Cmd, object):
                           self.ca_cert, self.user, self.ldap_password, True,
                           self.client_connect_timeout_ms, self.verbose,
                           use_http_base_transport=False, http_path=self.http_path,
-                          http_cookie_names=None)
+                          http_cookie_names=None, value_converter=value_converter)
       elif protocol == 'hs2-http':
         return StrictHS2Client(self.impalad, self.fetch_size, self.kerberos_host_fqdn,
                           self.use_kerberos, self.kerberos_service_name, self.use_ssl,
                           self.ca_cert, self.user, self.ldap_password, self.use_ldap,
                           self.client_connect_timeout_ms, self.verbose,
                           use_http_base_transport=True, http_path=self.http_path,
-                          http_cookie_names=self.http_cookie_names)
+                          http_cookie_names=self.http_cookie_names,
+                          value_converter=value_converter)
     if protocol == 'hs2':
       return ImpalaHS2Client(self.impalad, self.fetch_size, self.kerberos_host_fqdn,
                           self.use_kerberos, self.kerberos_service_name, self.use_ssl,
                           self.ca_cert, self.user, self.ldap_password, self.use_ldap,
                           self.client_connect_timeout_ms, self.verbose,
                           use_http_base_transport=False, http_path=self.http_path,
-                          http_cookie_names=None)
+                          http_cookie_names=None, value_converter=value_converter)
     elif protocol == 'hs2-http':
       return ImpalaHS2Client(self.impalad, self.fetch_size, self.kerberos_host_fqdn,
                           self.use_kerberos, self.kerberos_service_name, self.use_ssl,
@@ -619,7 +629,8 @@ class ImpalaShell(cmd.Cmd, object):
                           self.client_connect_timeout_ms, self.verbose,
                           use_http_base_transport=True, http_path=self.http_path,
                           http_cookie_names=self.http_cookie_names,
-                          http_socket_timeout_s=self.http_socket_timeout_s)
+                          http_socket_timeout_s=self.http_socket_timeout_s,
+                          value_converter=value_converter)
     elif protocol == 'beeswax':
       return ImpalaBeeswaxClient(self.impalad, self.fetch_size, self.kerberos_host_fqdn,
                           self.use_kerberos, self.kerberos_service_name, self.use_ssl,
@@ -1916,10 +1927,24 @@ def get_intro(options):
   if options.protocol == 'beeswax':
     intro += ("\n\nWARNING: The beeswax protocol is deprecated and will be removed in a "
               "future version of Impala.")
+    if options.hs2_fp_format:
+      intro += ("\n\nWARNING: Formatting floating-point values is "
+                "not supported with Beeswax protocol")
 
   return intro
 
 
+def _validate_hs2_fp_format_specification(format_specification):
+  try:
+    format_str = "{:%s}" % format_specification
+    format_str.format(float())
+  except UnicodeDecodeError as e:
+    raise e
+  except (ValueError, TypeError) as e:
+    raise FatalShellException(e)
+
+
+
 def impala_shell_main():
   """
   There are two types of options: shell options and query_options. Both can be set on the
@@ -2022,6 +2047,18 @@ def impala_shell_main():
           "mechanism (-l)", file=sys.stderr)
     raise FatalShellException()
 
+  if options.hs2_fp_format:
+    try:
+      _validate_hs2_fp_format_specification(options.hs2_fp_format)
+    except FatalShellException as e:
+      print("Invalid floating point format specification: %s" %
+            options.hs2_fp_format, file=sys.stderr)
+      raise e
+    except UnicodeDecodeError as e:
+      print("Unicode character in format specification is not supported",
+            file=sys.stderr)
+      raise FatalShellException(e)
+
   start_msg = "Starting Impala Shell"
 
   py_version_msg = "using Python {0}.{1}.{2}".format(
diff --git a/shell/impala_shell_config_defaults.py b/shell/impala_shell_config_defaults.py
index 513a16af7..2786fa8c3 100644
--- a/shell/impala_shell_config_defaults.py
+++ b/shell/impala_shell_config_defaults.py
@@ -58,4 +58,5 @@ impala_shell_defaults = {
             'http_socket_timeout_s': None,
             'global_config_default_path': '/etc/impalarc',
             'strict_hs2_protocol': False,
+            'fp_format_specification': None
     }
diff --git a/shell/make_shell_tarball.sh b/shell/make_shell_tarball.sh
index e696d7795..b3ae5e16e 100755
--- a/shell/make_shell_tarball.sh
+++ b/shell/make_shell_tarball.sh
@@ -152,6 +152,7 @@ cp ${SHELL_HOME}/ImpalaHttpClient.py ${TARBALL_ROOT}/lib
 cp ${SHELL_HOME}/shell_exceptions.py ${TARBALL_ROOT}/lib
 cp ${SHELL_HOME}/shell_output.py ${TARBALL_ROOT}/lib
 cp ${SHELL_HOME}/cookie_util.py ${TARBALL_ROOT}/lib
+cp ${SHELL_HOME}/value_converter.py ${TARBALL_ROOT}/lib
 cp ${SHELL_HOME}/impala-shell ${TARBALL_ROOT}
 cp ${SHELL_HOME}/impala_shell.py ${TARBALL_ROOT}
 cp ${SHELL_HOME}/compatibility.py ${TARBALL_ROOT}
diff --git a/shell/option_parser.py b/shell/option_parser.py
index beee7f412..abfc1794c 100755
--- a/shell/option_parser.py
+++ b/shell/option_parser.py
@@ -321,7 +321,12 @@ def get_option_parser(defaults):
                     "returned in an http response by the server or an intermediate proxy "
                     "then it will be included in each subsequent request for the same "
                     "connection.")
-
+  parser.add_option("--hs2_fp_format", type="str",
+                    dest="hs2_fp_format", default=None,
+                    help="Sets the printing format specification for floating point "
+                    "values when using HS2 protocol. The default behaviour makes the  "
+                    "values handled by Python's str() built-in method. Use '%16G' to "
+                    "match Beeswax protocol's floating-point output format")
 
   # add default values to the help text
   for option in parser.option_list:
diff --git a/shell/packaging/make_python_package.sh b/shell/packaging/make_python_package.sh
index 2ccb9a420..954f7eae5 100755
--- a/shell/packaging/make_python_package.sh
+++ b/shell/packaging/make_python_package.sh
@@ -60,6 +60,7 @@ assemble_package_files() {
   cp "${SHELL_HOME}/ImpalaHttpClient.py" "${MODULE_LIB_DIR}"
   cp "${SHELL_HOME}/shell_exceptions.py" "${MODULE_LIB_DIR}"
   cp "${SHELL_HOME}/cookie_util.py" "${MODULE_LIB_DIR}"
+  cp "${SHELL_HOME}/value_converter.py" "${MODULE_LIB_DIR}"
 
   cp "${SHELL_HOME}/packaging/README.md" "${PACKAGE_DIR}"
   cp "${SHELL_HOME}/packaging/MANIFEST.in" "${PACKAGE_DIR}"
diff --git a/shell/value_converter.py b/shell/value_converter.py
new file mode 100644
index 000000000..92a4259c4
--- /dev/null
+++ b/shell/value_converter.py
@@ -0,0 +1,51 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from TCLIService.TCLIService import TTypeId
+
+
+class ValueConverter(object):
+
+  def get_converter(value):
+      pass
+
+  def override_floating_point_converter(format_specification):
+      pass
+
+
+class HS2ValueConverter(ValueConverter):
+
+  def __init__(self):
+      self.value_converters = {
+          TTypeId.BOOLEAN_TYPE: lambda b: 'true' if b else 'false',
+          TTypeId.TINYINT_TYPE: str,
+          TTypeId.SMALLINT_TYPE: str,
+          TTypeId.INT_TYPE: str,
+          TTypeId.BIGINT_TYPE: str,
+          TTypeId.BINARY_TYPE: str,
+          TTypeId.FLOAT_TYPE: str,
+          TTypeId.DOUBLE_TYPE: str
+      }
+
+  def get_converter(self, value):
+      return self.value_converters.get(value, None)
+
+  def override_floating_point_converter(self, format_specification):
+      def convert(value):
+          return ('{:%s}' % format_specification).format(value)
+      self.value_converters[TTypeId.FLOAT_TYPE] = convert
+      self.value_converters[TTypeId.DOUBLE_TYPE] = convert
diff --git a/tests/shell/test_shell_commandline.py b/tests/shell/test_shell_commandline.py
index 5d334c30e..33b4bc2dc 100644
--- a/tests/shell/test_shell_commandline.py
+++ b/tests/shell/test_shell_commandline.py
@@ -1305,3 +1305,81 @@ class TestImpalaShell(ImpalaTestSuite):
       assert len(lines) >= 2
       assert "1\n" in lines[len(lines) - 2]
       assert "Fetched 1 row(s)" in lines[len(lines) - 1]
+
+  def skip_if_protocol_is_beeswax(self, vector):
+    """Helper to skip Beeswax protocol on formatting tests"""
+    if vector.get_value("protocol") == "beeswax":
+      pytest.skip("Floating-point value formatting is not supported "
+                  "with Beeswax")
+
+  def validate_fp_format(self, vector, column_type, format, value, expected_values):
+    args = ['--hs2_fp_format', format, '-q',
+           'select cast("%s" as %s) as fp_value' % (value, column_type)]
+    result = run_impala_shell_cmd(vector, args)
+
+    assert(any(expected_value in result.stdout for expected_value in expected_values))
+
+  def test_hs2_fp_format_types(self, vector):
+    """Tests formatting with double and float value"""
+    self.skip_if_protocol_is_beeswax(vector)
+    self.validate_fp_format(vector, column_type='double', format='.16f',
+                            value='0.1234567891011121314',
+                            expected_values='0.1234567891011121')
+
+    expected_values_float = ['0.12345679104328156', '0.12345679000000000']
+    self.validate_fp_format(vector, column_type='float', format='.17f',
+                            value='0.1234567891011121314',
+                            expected_values=expected_values_float)
+
+  def test_hs2_fp_format_modifiers(self, vector):
+    """Test formatting with various modifiers, like mode, grouping, sign"""
+    self.skip_if_protocol_is_beeswax(vector)
+    self.validate_fp_format(vector, column_type='double', format='+f',
+                            value='123456789123456789',
+                            expected_values='+123456789123456784.000000')
+
+    self.validate_fp_format(vector, column_type='double', format='+.16f',
+                            value='-12.3456789123456789',
+                            expected_values='-12.3456789123456794')
+
+    expected_grouped_value = '+123,456,788,999,999,994,034,486,658,482,569,216.000000'
+    self.validate_fp_format(vector, column_type='double', format='+,f',
+                            value='1.23456789e+35',
+                            expected_values=expected_grouped_value)
+
+    self.validate_fp_format(vector, column_type='double', format='+E',
+                            value='12345678912345',
+                            expected_values='+1.234568E+13')
+
+  def test_hs2_fp_format_nan_inf_null(self, vector):
+    """Test NaN, inf, null value formatting"""
+    self.skip_if_protocol_is_beeswax(vector)
+    self.validate_fp_format(vector, column_type='double', format='F',
+                            value='NaN',
+                            expected_values='NAN')
+
+    self.validate_fp_format(vector, column_type='double', format='F',
+                            value='-inf',
+                            expected_values=['-INF', 'NULL'])
+
+    self.validate_fp_format(vector, column_type='double', format='F',
+                            value='NULL',
+                            expected_values='NULL')
+
+  def test_hs2_fp_format_invalid(self, vector):
+    """Test invalid format specification"""
+    self.skip_if_protocol_is_beeswax(vector)
+    error_message = 'Invalid floating point format specification: invalid'
+    args = ['--hs2_fp_format', 'invalid']
+    result = run_impala_shell_cmd(vector, args, expect_success=False)
+
+    assert error_message in result.stderr
+
+  def test_fp_default(self, vector):
+    """Test default floating point value formatting"""
+    expected_py2 = '0.123456789101'
+    expected_py3 = '0.1234567891011121'
+    args = ['-q', 'select cast("0.1234567891011121314" as double) as fp_value', '-B']
+    result = run_impala_shell_cmd(vector, args)
+
+    assert expected_py2 in result.stdout or expected_py3 in result.stdout


[impala] 02/02: IMPALA-11599/IMPALA-11605: GCC 10: Fix gdb and change to flatbuffers 1.9

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

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

commit 296e94411d3344e2969d4b083036ff238e80ad19
Author: Joe McDonnell <jo...@cloudera.com>
AuthorDate: Thu Sep 22 15:35:45 2022 -0700

    IMPALA-11599/IMPALA-11605: GCC 10: Fix gdb and change to flatbuffers 1.9
    
    This contains two fixes for issues introduced by
    IMPALA-9999's upgrade to GCC 10:
    IMPALA-11599: Builds toolchain gdb with the OS compiler,
    fixing an issue where it could not run on Centos 7 due
    to problems with its RPATH / finding appropriate libstdc++.
    IMPALA-11605: Reverts to flatbuffers 1.9, which avoids
    a mismatch with the version that Hive uses. This avoids
    classpath incompatibility issues.
    
    Testing:
     - Verified that toolchain gdb works on Centos 7
     - Verified that interoperability with Hive is restored
    
    Change-Id: Ieaafdb1da05cab60b254115faaae55dba8b67dfd
    Reviewed-on: http://gerrit.cloudera.org:8080/19034
    Tested-by: Joe McDonnell <jo...@cloudera.com>
    Reviewed-by: Wenzhe Zhou <wz...@cloudera.com>
---
 bin/impala-config.sh | 4 ++--
 fe/pom.xml           | 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/bin/impala-config.sh b/bin/impala-config.sh
index b4f2e5d85..34034137c 100755
--- a/bin/impala-config.sh
+++ b/bin/impala-config.sh
@@ -77,7 +77,7 @@ export IMPALA_VERSION=4.2.0-SNAPSHOT
 # moving to a different build of the toolchain, e.g. when a version is bumped or a
 # compile option is changed. The build id can be found in the output of the toolchain
 # build jobs, it is constructed from the build number and toolchain git hash prefix.
-export IMPALA_TOOLCHAIN_BUILD_ID=215-9ec4f91d2d
+export IMPALA_TOOLCHAIN_BUILD_ID=223-7cf7e75bc8
 # Versions of toolchain dependencies.
 # -----------------------------------
 export IMPALA_AVRO_VERSION=1.7.4-p5
@@ -100,7 +100,7 @@ export IMPALA_CURL_VERSION=7.78.0
 unset IMPALA_CURL_URL
 export IMPALA_CYRUS_SASL_VERSION=2.1.23
 unset IMPALA_CYRUS_SASL_URL
-export IMPALA_FLATBUFFERS_VERSION=1.12.0
+export IMPALA_FLATBUFFERS_VERSION=1.9.0-p1
 unset IMPALA_FLATBUFFERS_URL
 export IMPALA_GCC_VERSION=10.4.0
 unset IMPALA_GCC_URL
diff --git a/fe/pom.xml b/fe/pom.xml
index 863f285f8..ba69bd5d0 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -351,7 +351,7 @@ under the License.
     <dependency>
       <groupId>com.google.flatbuffers</groupId>
       <artifactId>flatbuffers-java</artifactId>
-      <version>1.12.0</version>
+      <version>1.9.0</version>
     </dependency>
 
     <dependency>