Impala 4.4 Change Log
Release Notes - IMPALA - Version Impala 4.4.0
New Feature
Improvement
- [IMPALA-3825] - Distribute runtime filter aggregation across cluster
- [IMPALA-5081] - Expose IR optimization level via query option
- [IMPALA-8675] - Remove db/table count metrics from impalad in LocalCatalog mode
- [IMPALA-9086] - Show Hive configurations in /hadoop-varz page
- [IMPALA-9375] - Remove DirectMetaProvider usage from CatalogMetaProvider
- [IMPALA-10048] - Speed up dumping breadpad symbols in bin/jenkins/finalize.sh
- [IMPALA-10949] - Improve batching logic of events
- [IMPALA-11123] - Optimize count(*) for ORC scans
- [IMPALA-11553] - Add events specific metrics on table level
- [IMPALA-11972] - Factor in row width during ProcessingCost calculation.
- [IMPALA-12018] - Consider runtime filters in resource estimates
- [IMPALA-12076] - Potential performance improvement using ZSTD's ZSTD_decompressDCtx interface
- [IMPALA-12125] - dump_breakpad_symbols.py should support OS packages without separate debug packages
- [IMPALA-12308] - Implement DIRECTED distribution mode for Iceberg tables
- [IMPALA-12318] - Use spnego dedicated keytab
- [IMPALA-12362] - Improve Linux packaging support.
- [IMPALA-12373] - Implement Small String Optimization for StringValue
- [IMPALA-12385] - Enable Periodic metrics by default
- [IMPALA-12390] - Enable performance related clang-tidy checks
- [IMPALA-12399] - Pass eventTypeSkipList with OPEN_TXN in NotificationEventRequest to avoid receiving OPEN_TXN events from HMS
- [IMPALA-12402] - Make CatalogdMetaProvider's cache concurrency level configurable
- [IMPALA-12443] - Add catalog timeline for all DDL profiles
- [IMPALA-12460] - Add lag and histogram of event processing in the log
- [IMPALA-12462] - Do not update unchanged partitions in COMPUTE STATS
- [IMPALA-12463] - Allow batching of non consecutive metastore events
- [IMPALA-12482] - StatestoreSubscriber.UpdateState logging is too noisy at VLOG level 2
- [IMPALA-12487] - Skip reloading file metadata for ALTER_TABLE events with trivial changes in StorageDescriptor
- [IMPALA-12505] - Define a new impala flag that runs the trusted domain check on the origin if the trusted_domain_use_xff_header flag is enabled and no XFF header is received
- [IMPALA-12509] - Optimize the backend startup and planner time of large Iceberg table query
- [IMPALA-12518] - Combine all exec_option dimension in test_vector.py
- [IMPALA-12540] - SQL Interface to Running Queries/DDLs/DMLs
- [IMPALA-12544] - Replace scan progress with query progress as progress reporting for the shell
- [IMPALA-12626] - Include List of Referenced Tables/Views in Query Log Table
- [IMPALA-12631] - Improve count star performance for parquet scans
- [IMPALA-12654] - Turn query_cpu_count_divisor flag into a query option
- [IMPALA-12657] - Improve ProcessingCost of ScanNode and NonGroupingAggregator
- [IMPALA-12668] - Turn on Clang Tidy's clang-diagnostic-implicit-fallthrough check
- [IMPALA-12726] - Simulate large scale query planning in TpcdsCpuCostPlannerTest
- [IMPALA-12727] - Reduce the DiskIoMgr thread parameters for filesystems that are not being tested
- [IMPALA-12729] - Allow creating primary keys for Iceberg tables
- [IMPALA-12730] - Avoid Weverything for Clang Tidy
- [IMPALA-12782] - Show info of the event processing in /events webUI
- [IMPALA-12810] - Simplify IcebergDeleteNode and IcebergDeleteBuilder
- [IMPALA-12834] - Add query load information to the query profile
- [IMPALA-12838] - Add exec_options parameter to single_node_perf_run.py
- [IMPALA-12847] - Make computeScanRangeLocations and computeStats idempotent in HdfsScanNode.java
- [IMPALA-12874] - Identify Catalog HA and StateStore HA from the web debug endpoint
- [IMPALA-12881] - Use JoinNode.getFkPkJoinCardinality in reduceCardinalityForScanNode
- [IMPALA-12977] - add search and pagination to /hadoop-varz
- [IMPALA-12980] - Translate CpuAsk into admission control slot to use
- [IMPALA-12988] - Calculate an unbounded version of CpuAsk
- [IMPALA-13016] - Fix ambiguous row_regex that check for no-existence
Bug
- [IMPALA-4545] - test_decimal_casting.py has invalid test vector under "pairwise" exploration strategy
- [IMPALA-10120] - Beeline hangs when connecting to coordinators
- [IMPALA-10987] - Changing impala.disableHmsSync in Hive can break event processing
- [IMPALA-11068] - Query hit OOM under high decompression activity
- [IMPALA-11499] - Refactor UrlEncode function to handle special characters
- [IMPALA-11501] - Add flag to allow metadata-cache operations on masked tables
- [IMPALA-11542] - TestFailpoints::test_failpoints crash in ARM build
- [IMPALA-11568] - Backend test TimeSeriesCounterToJsonTest in runtime-profile-test is flaky
- [IMPALA-11710] - Table properties are not updated in Iceberg metadata files
- [IMPALA-11762] - [DOCS] Reserved words documentation lags behind the code
- [IMPALA-11805] - Codegen cache size estimation is less than the actual allocation
- [IMPALA-11909] - bin/jenkins/finalize.sh should use an absolute path for resolve_minidump.py
- [IMPALA-11921] - test_large_sql seems to be flaky
- [IMPALA-11938] - Raise error when NUM_NODES is set to a value other than 0 or 1
- [IMPALA-12054] - Some of the non-kudu tests fail if kudu cluster is not running
- [IMPALA-12072] - Include snapshot id of Iceberg tables in query plan / profile
- [IMPALA-12083] - TestTpcdsQueryWithProcessingCost.test_tpcds_q67a intermmitently failed by Memory limit exceeded
- [IMPALA-12356] - Partition created by INSERT will make the next ALTER_PARTITION event on it always treated as self-event
- [IMPALA-12371] - Add better cardinality estimation for Iceberg V2 tables with deletes
- [IMPALA-12397] - NullPointerException in SHOW ROLES when there are no roles
- [IMPALA-12398] - Ranger role not exists when altering db/table/view owner to a role
- [IMPALA-12429] - TestTpcdsQueryWithProcessingCost.test_tpcds_q51a and TestTpcdsQuery.test_tpcds_q67a failed
- [IMPALA-12444] - PROCESSING_COST_MIN_THREADS can get ignored by scan fragment.
- [IMPALA-12445] - Native toolchain's redhat9 docker image doesn't have the "file" utility
- [IMPALA-12449] - SSE2NEON is slow compared to native NEON code
- [IMPALA-12464] - CatalogdMetaPRovider triggers CannotAccessFieldException on JDK17 during custom_cluster tests
- [IMPALA-12474] - Latest metastore event id polling can be blocked
- [IMPALA-12483] - Add new api in MetastoreServiceHandler for update_transaction_statistics method
- [IMPALA-12492] - Add catalog metrics for total pending events and lag time of the event-processor
- [IMPALA-12493] - Impala Query cancelled while Analyzing or Compiling partially closes but query remains on Coordinator
- [IMPALA-12496] - query_test.test_ext_data_sources.TestExtDataSources fails on S3/Ozone
- [IMPALA-12499] - TestScanMemLimit.test_hdfs_scanner_thread_mem_scaling fails intermittently
- [IMPALA-12500] - TestObservability.test_global_exchange_counters is flaky
- [IMPALA-12510] - ProcessingCost.getNumInstanceMax() should not return 0.
- [IMPALA-12511] - Confusing timeout logs due to THRIFT-5670
- [IMPALA-12513] - The metadata should be reset when the CatalogD become active
- [IMPALA-12517] - impala-shell with Python 3 mangles binary output
- [IMPALA-12524] - Duplicate TestTextScanRangeLengths in test_scanners.py
- [IMPALA-12525] - statestore.active-status did not reach value True in 120s
- [IMPALA-12526] - BackendConfig.INSTANCE could be null in the frontend test testResetMetadataDesc
- [IMPALA-12528] - test_hdfs_scanner_thread_non_reserved_bytes could occasionally fail
- [IMPALA-12529] - Add warning logs if acquiring table lock takes long
- [IMPALA-12532] - change_database should happen before set_configuration in cancel_util.py
- [IMPALA-12535] - Wrong unit of thread duration shown in the json result of /thread-group?all&json URL
- [IMPALA-12543] - test_iceberg_self_events failed in JDK11 build
- [IMPALA-12548] - Clarify the meaning of AGG_MEM_CORRELATION_FACTOR
- [IMPALA-12550] - test_statestored_auto_failover_with_disabling_network flaky
- [IMPALA-12552] - impala-shell should not call encode on kerberos_host_fqdn in python 3 env
- [IMPALA-12555] - bin/jenkins/populate_m2_directory.py fails during bootstrap in recent builds
- [IMPALA-12556] - test_two_statestored_with_force_active fails or flaky
- [IMPALA-12557] - DELETE throws DateTimeParseException when deleting from DAY-partitioned Iceberg table
- [IMPALA-12561] - Event-processor shouldn't go into ERROR state for failures in fetching events
- [IMPALA-12564] - TSAN test run fails to start Hive Server on Ubuntu 22.04
- [IMPALA-12565] - When executing pmod (anyNumber, 0), it will cause the executor to crash
- [IMPALA-12571] - Data race due to StringValue::Smallify of template tuple
- [IMPALA-12575] - test_executor_group_num_queries_executing_metric fails in UBSAN build
- [IMPALA-12577] - last-synced-event-time is not updated when events are filtered out
- [IMPALA-12579] - Add logs for loading table list in catalogd startup
- [IMPALA-12581] - ILIKE and IREGEXP don't work with non-const argument values
- [IMPALA-12582] - Executors crashed while generating the runtime filters
- [IMPALA-12584] - Add backend config to restrict data file locations for Iceberg tables
- [IMPALA-12589] - NPE on querying external tables pointing to a single file
- [IMPALA-12590] - Precommit build script cannot execute dmesg to check for OOM-kills on Ubuntu 20.04
- [IMPALA-12593] - query_test.test_iceberg.TestIcebergTable.test_drop_partition fails on S3
- [IMPALA-12595] - Allow automatic removal of old logs from previous PID
- [IMPALA-12602] - Timed out queries are not unregistered until session is closed
- [IMPALA-12606] - Sporadic failures around query_test.test_queries.TestQueries.test_intersect
- [IMPALA-12617] - Statestore hit DCHECK failure in ThreadPool::Offer()
- [IMPALA-12628] - iceberg negative test fail saying can't connect but there is no obvious server crash
- [IMPALA-12629] - TestRanger fails in test_column_masking and test_row_filtering
- [IMPALA-12630] - TestOrcStats.test_orc_stats fails in count-start on lineitem with filter
- [IMPALA-12632] - TSAN detected data race accessing CpuUsageRatios
- [IMPALA-12633] - DCHECK failure on query_handle->is_inflight()
- [IMPALA-12634] - DCHECK failure in ~ClientRequestState destructor
- [IMPALA-12635] - Tests fail with "Memory is likely oversubscribed"
- [IMPALA-12640] - Remove IcebergDeleteSink
- [IMPALA-12643] - resolve_minidumps.py's minidump_stackwalk call hits OOM on Centos7
- [IMPALA-12655] - PlannerTest.testProcessingCost seems flaky
- [IMPALA-12658] - UPDATE Iceberg table FROM view throws IllegalStateException
- [IMPALA-12661] - ASAN heap-use-after-free in IcebergMetadataScanNode
- [IMPALA-12665] - Adjust complete_micro_batch_ length to new scratch_batch_->capacity after ScratchTupleBatch::Reset
- [IMPALA-12666] - Statestore topic metrics are wrong when impalads are shutdown
- [IMPALA-12670] - CatalogdMetaProvider.getIfPresent() not throwing the underlying InconsistentMetadataFetchException
- [IMPALA-12673] - Iceberg table migraton fails for '/' in partition values
- [IMPALA-12676] - UBSAN intermittent int overflow on internal-server-test
- [IMPALA-12678] - test_skipping_batching_events intermittent failure
- [IMPALA-12681] - Some local file descriptors not released when using remote spilling
- [IMPALA-12683] - Incorrect last synced event time in testDisableEventSyncFlag
- [IMPALA-12687] - Key conflicts in catalog inFlightOperations when catalog RPC retries
- [IMPALA-12689] - Toolchain TPC-H and TPC-DS binaries are not built with optimizations
- [IMPALA-12692] - Typo in docs about random() function
- [IMPALA-12693] - Typo in link for ltrim in string functions docs
- [IMPALA-12698] - TestScratchDir.test_scratch_dirs_batch_reading seems flaky
- [IMPALA-12699] - Coordinator should retry GetPartialCatalogObject request and apply a recv timeout
- [IMPALA-12702] - Est. #Rows in ExecSummary does not reflect reduction from runtime filter.
- [IMPALA-12703] - ExchangeNode should use getFilteredCardinality for costing.
- [IMPALA-12704] - NullPointerException occurs when quering metadata of empty Iceberg table
- [IMPALA-12708] - An UPDATE creates 2 new snapshots in Iceberg tables
- [IMPALA-12711] - DDL/DML errors are not shown in impalad logs
- [IMPALA-12714] - test_reduced_cardinality_by_filter is failing on non HDFS builds
- [IMPALA-12716] - Failing TestWebPage.test_catalog_operations_with_rpc_retry
- [IMPALA-12718] - trim() functions are lack of utf-8 support
- [IMPALA-12719] - Reload file metadata for truncate op in metastore
- [IMPALA-12721] - TestScratchDir.test_scratch_dirs_remote_spill seems flaky
- [IMPALA-12740] - TestHdfsJsonScanNodeErrors fails in exhaustive mode
- [IMPALA-12742] - DELETE/UPDATE Iceberg table partitioned by DATE fails with error
- [IMPALA-12743] - Incremental stats are filtered out by HMS due to HIVE-27114
- [IMPALA-12745] - dump_breakpad_symbols.py's parallelism doesn't work with RPM/DEBs
- [IMPALA-12747] - internal-server-test crashes with SIGSEGV
- [IMPALA-12762] - Build package from scratch failed by cmake error
- [IMPALA-12765] - Balance consecutive partitions better for Iceberg tables
- [IMPALA-12770] - ExprRewriter enter infinite loop for nested Case expressions
- [IMPALA-12778] - Catalog operations page is showing UTC time strings
- [IMPALA-12784] - ScanNode. getFilteredInputCardinality is buggy.
- [IMPALA-12787] - Concurrent DELETE and UPDATE operations on Iceberg tables can be problematic
- [IMPALA-12788] - HBaseTable still get loaded even if HBase is down
- [IMPALA-12790] - ScanNode.getInputCardinality can overestimate if LIMIT is large.
- [IMPALA-12796] - Several tests in test_scanners.py failed by wrong row counts
- [IMPALA-12798] - regr_intercept() might have inconsistant results on TIMESTAMP values
- [IMPALA-12808] - test_iceberg_deletes_and_updates failed by timeout error
- [IMPALA-12811] - Exception during re-analyze can be lost
- [IMPALA-12814] - VERIFY_IS_NOT_IN does not work with row_regex:
- [IMPALA-12816] - Remove exclusion of Netty to allow SSL connections to HBase.
- [IMPALA-12821] - test_create_table_profile_events fails if event-processor is stopped
- [IMPALA-12827] - Precondition was hit in MutableValidReaderWriteIdList
- [IMPALA-12830] - test_webserver_hide_logs_link() could fail in the exhaustive build
- [IMPALA-12831] - HdfsTable.toMinimalTCatalogObject() should hold table read lock to generate incremental updates
- [IMPALA-12832] - EventProcessor shouldn't stop for failures in single-table event
- [IMPALA-12833] - Enabled starting flag 'catalogd_ha_reset_metadata_on_failover' by default
- [IMPALA-12840] - test_reuse_partitions_transactional is flaky
- [IMPALA-12842] - JS_TEST fails on ARM instances on node.js binaries installed for wrong CPU architecture
- [IMPALA-12849] - Standby catalogd should reject requests from coordinators
- [IMPALA-12851] - Process AllocWriteIdEvent do not add txnId to tableWriteIds mapping in Catalog
- [IMPALA-12855] - NullPointerException in firing RELOAD events if the partition is just dropped
- [IMPALA-12859] - Most steps fail in internal-server-test during some build types
- [IMPALA-12860] - Invoke validateDataFilesExist for RowDelta operations
- [IMPALA-12878] - TestResultSpoolingCancellation.test_cancellation failed in UBSAN build
- [IMPALA-12879] - Conjunct not referring to table field causes ERROR for Iceberg table
- [IMPALA-12894] - Optimized count(*) for Iceberg gives wrong results after a Spark rewrite_data_files
- [IMPALA-12903] - Querying virtual column FILE__POSITION for TEXT and JSON tables crashes Impala
- [IMPALA-12904] - test_type_conversions_hive3 silently passes because of wrongly defined test dimensions
- [IMPALA-12915] - double-free memory issue in unifiedbetests if it's built in dynamically linked mode
- [IMPALA-12929] - Skip loading HDFS permissions in local-catalog mode
- [IMPALA-12930] - TestExtDataSources.test_jdbc_data_source fails in some builds
- [IMPALA-12933] - Catalogd should set eventTypeSkipList when fetching specifit events for a table
- [IMPALA-12938] - test_no_inaccessible_objects failed in JDK11 build
- [IMPALA-12942] - test_virtual_column_file_position_generic flaky
- [IMPALA-12944] - test_query_log_table_different_table failed in Ozone build
- [IMPALA-12953] - CentOS 8 builds fail with python ImportError: No module named RuntimeProfile.ttypes
- [IMPALA-12960] - Incorrect RowsPassedThrough Metric in Streaming Aggregation
- [IMPALA-12963] - Testcase test_query_log_table_lower_max_sql_plan failed in ubsan builds
- [IMPALA-12969] - DeserializeThriftMsg may leak JNI resources
- [IMPALA-12970] - Test failure at test_read_equality_deletes in test_iceberg in exhaustive build
- [IMPALA-12989] - LICENSE and NOTICE files are missing in DEB/RPM packages
- [IMPALA-12991] - Eliminate unnecessary SORT for Iceberg DELETEs
- [IMPALA-12998] - Query History fails DCHECK with unknown DDL type
- [IMPALA-12999] - Missing log4j.properties in the package
- [IMPALA-13002] - Iceberg V2 tables with Avro delete files aren't read properly
- [IMPALA-13003] - Server exits early failing to create impala_query_log with AlreadyExistsException
- [IMPALA-13004] - heap-use-after-free error in ExprTest AiFunctionsTest
- [IMPALA-13005] - Query Live table not visible via metastore
- [IMPALA-13009] - Potential leak of partition deletions in the catalog topic
- [IMPALA-13012] - Completed queries write fails regularly under heavy load
- [IMPALA-13015] - Dataload fails due to concurrency issue with test.jceks
- [IMPALA-13020] - catalog-topic updates >2GB do not work due to Thrift's max message size
- [IMPALA-13024] - Several tests timeout waiting for admission
- [IMPALA-13054] - EnqueueCompletedQuery very slow on deeply nested plans
- [IMPALA-13061] - Query Live table fails to load if default_transactional_type=insert_only set globally
Task
- [IMPALA-11157] - Build native Hadoop libraries for aarch64
- [IMPALA-12115] - Must rebuild minicluster to switch filesystems
- [IMPALA-12279] - Bump CDP_BUILD_NUMBER for Iceberg 1.3
- [IMPALA-12334] - Enable debug symbols in LLVM library debug build
- [IMPALA-12354] - Use aarch64 native-toolchain builds
- [IMPALA-12355] - Produce arm64 container images
- [IMPALA-12434] - Isolate shell/pkg_resources.py to keep it off of the PYTHONPATH
- [IMPALA-12442] - Avoid running stress tests twice
- [IMPALA-12453] - TestIcebergV2Table::test_delete_partitioned slows down then times out in exhaustive mode
- [IMPALA-12480] - Match hadoop-aliyun to hadoop version
- [IMPALA-12484] - Update Kudu for new libunwind
- [IMPALA-12485] - Remove Python scripts use of has_key
- [IMPALA-12490] - Fix tests on ARM
- [IMPALA-12514] - Update jetty-server, make it test-only
- [IMPALA-12515] - Allow building impala-shell tarballs for multiple Python 3 versions
- [IMPALA-12516] - HDFS in minicluster fails to use cache with RHEL 8 on ARM
- [IMPALA-12563] - ARM UBSAN build fails with undefined reference to aarch symbols
- [IMPALA-12566] - RpcMgrKerberizedTest cases fail on RedHat 8
- [IMPALA-12618] - Update README.md to reduce emphasis on Hadoop
- [IMPALA-12697] - Native toolchain upload to S3 silently failed
- [IMPALA-12746] - Bump jackson-databind version to 2.15
- [IMPALA-12767] - Upgrade Guava to 32.0.1 due to CVE-2023-2976
- [IMPALA-12801] - Increase default query_log_size
- [IMPALA-12818] - Implement the initial framework for caching tuples
- [IMPALA-12819] - InaccessibleObjectException found during LocalCatalogTest
- [IMPALA-12868] - Catalogd's statestore subscriber gets stuck in a loop trying to process UpdateCatalogd RPC
- [IMPALA-12883] - Add support for changing the charge for a cache entry
- [IMPALA-12900] - Compile binutils with -O3 in the toolchain
- [IMPALA-12905] - Implement disk-based tuple caching
- [IMPALA-12920] - Support ai_generate_text built-in function for OpenAI's LLMs
- [IMPALA-12924] - Upgrade xmlsec to address CVE in 2.2.3
- [IMPALA-13045] - Fix intermittent failure in TestQueryLive.test_local_catalog
Sub-task
- [IMPALA-7131] - Support external data sources in local catalog mode
- [IMPALA-11996] - Iceberg Metadata querying executor change
- [IMPALA-12038] - Switch report_benchmark_results.py to Python 3
- [IMPALA-12205] - Add support to STRUCT type Iceberg Metadata table columns
- [IMPALA-12375] - DataSource objects are not persistent
- [IMPALA-12376] - DataSourceScanNode drop some returned rows if FLAGS_data_source_batch_size is greater than default value
- [IMPALA-12377] - Improve count star performance for external data source
- [IMPALA-12378] - Auto Ship Data Source library in Impala
- [IMPALA-12379] - Detect available jdbc drivers without restarting Impala
- [IMPALA-12380] - Securing dbcp.password for JDBC external data source
- [IMPALA-12381] - Add jdbc related properties to JDBC data source object
- [IMPALA-12431] - Support reading compressed JSON file
- [IMPALA-12470] - Support different schemes for jdbc driver url when creating external jdbc table
- [IMPALA-12471] - Scripts to run unit-tests of external jdbc table for MySQL
- [IMPALA-12495] - Describe command for Iceberg metadata tables
- [IMPALA-12501] - Caching of JDBC drivers in local storage for external JDBC tables
- [IMPALA-12502] - Support Impala to Impala federation
- [IMPALA-12527] - test_metadata_tables could occasionally fail in the s3 build
- [IMPALA-12530] - Allow Postgres server to accept remote connections from hosts in same subnet
- [IMPALA-12597] - Basic equality delete support
- [IMPALA-12600] - Support equality deletes when table has partition or schema evolution
- [IMPALA-12642] - Support setting query options for Impala in the JdbcStorageConfig
- [IMPALA-12653] - Update documentation about the UPDATE statement
- [IMPALA-12660] - TSAN error in ImpalaServer::QueryStateRecord::Init
- [IMPALA-12671] - LdapHS2Test.testImpalaExtJdbcTables failed for jdk17 builds
- [IMPALA-12793] - Change JDBC table creation syntax
- [IMPALA-12802] - Support ALTER TABLE for JDBC tables
- [IMPALA-12815] - Support timestamp data type for predicates for external data source table
- [IMPALA-12825] - Install thrift into the impala-python virtualenv rather than using the toolchain
- [IMPALA-12848] - TestExtDataSources::test_catalogd_ha_failover seems flaky
- [IMPALA-12896] - Avoid JDBC table to be set as transactional table
- [IMPALA-12910] - Run TPCH/TPCDS queries for external JDBC tables
- [IMPALA-12925] - Fix decimal data type for external JDBC table
- [IMPALA-12928] - Mask dbcp.password table property of JDBC table for 'desc formatted' and 'show create table' commands
Test
- [IMPALA-12542] - test_query_cancel_created failed in ASAN build
- [IMPALA-12567] - Deflake test_75_percent_availability
- [IMPALA-12601] - Add a fully partitioned TPC-DS dataset for planner tests
- [IMPALA-12690] - Expand PlannerTest.testProcessingCost to test all TPC-DS queries.
- [IMPALA-12777] - Fix tpcds/tpcds-q66.test
- [IMPALA-12864] - Deflake test_query_log_size_in_bytes
- [IMPALA-12898] - Tidy up test matrix of test_scanner.py
Documentation
- [IMPALA-11967] - Document that Compute Incremental Stats support specifying the column list
- [IMPALA-12491] - Document whether the remote data cache has TTL
- [IMPALA-12858] - idle_client_poll_period_s misnamed in documentation
- [IMPALA-13000] - Document OPTIMIZE TABLE