来源:Apache Hive官网
发布日期:2022-11-16
阅读次数:7419
评论:0
更新详情
Release Notes - Hive - Version 4.0.0-alpha-2
Sub-task
[HIVE-17370] - Some tests are failing with java.io.FileNotFoundException: File file:/tmp/hadoop/mapred/...
[HIVE-17943] - select ROW__ID, t, si, i from over10k_orc_bucketed where b = 4294967363 and t < 100 order by ROW__ID fails on LLAP
[HIVE-19930] - Resolve explain.out differences introduced by transactional metastore statistics
[HIVE-20070] - ptest optimization - Replicate ACID/MM tables write operations.
[HIVE-20387] - Move non-server related methods from Warehouse to MetastoreUtils
[HIVE-20532] - One of the task , either move or add partition can be avoided in repl load flow
[HIVE-20564] - Remove Hive Server dependency on Metastore Server
[HIVE-21108] - Assign writeId for stats update for a converted transactional table
[HIVE-22417] - Remove stringifyException from MetaStore
[HIVE-22748] - Remove disabling of IdentityProjectRemover transformation for Return Path
[HIVE-23745] - Avoid copying userpayload in task communicator
[HIVE-24941] - [Evaluate] if ReplicationSpec is needed for DataConnectors.
[HIVE-25091] - Implement connector provider for MSSQL and Oracle
[HIVE-25446] - Wrong exception thrown if capacity<=0
[HIVE-25556] - Remove com.vlkan.flatbuffers dependency from serde
[HIVE-25921] - Overwrite table metadata for bootstraped tables
[HIVE-26045] - Detect timed out connections for providers and auto-reconnect
[HIVE-26118] - [Standalone Beeline] Jar name mismatch between build and assembly
[HIVE-26171] - HMSHandler get_all_tables method can not retrieve tables from remote database
[HIVE-26183] - Create delete writer for the UPDATE statemens
[HIVE-26223] - Integrate ESRI GeoSpatial UDFs
[HIVE-26224] - Add support for ESRI GeoSpatial SERDE formats
[HIVE-26240] - Add tests for Esri UDFs for Geospatial
[HIVE-26278] - Add unit tests for Hive#getPartitionsByNames using batching
[HIVE-26279] - Drop unused requests from TestHiveMetaStoreClientApiArgumentsChecker
[HIVE-26285] - Overwrite database metadata on original source in optimised failover.
[HIVE-26299] - Drop data connector with argument ifNotExists(true) should not throw NoSuchObjectException
[HIVE-26301] - Fix ACID tables bootstrap during reverse replication in unplanned failover.
[HIVE-26316] - Handle dangling open txns on both src & tgt in unplanned failover.
[HIVE-26319] - Iceberg integration: Perform update split early
[HIVE-26361] - Merge statement compile fails if split update is off
[HIVE-26365] - Remove column statistics collection task from merge statement plan
[HIVE-26448] - Improve UnsignedInt128.setV*()
[HIVE-26488] - Fix NPE in DDLSemanticAnalyzerFactory during compilation
[HIVE-26512] - ST_GeometryProcessing is incorrectly registered as function
[HIVE-26541] - WebHCatServer start fails with NPE
[HIVE-26549] - WebHCat servers fails to start due to authentication filter configuration
Bug
[HIVE-5134] - add tests to partition filter JDO pushdown for like and make sure it works, or remove it
[HIVE-14981] - Eliminate unnecessary MapJoin restriction in HIVE-11394
[HIVE-17162] - get rid of "skipCorrupt" flag
[HIVE-17539] - User impersonation failure is not propagated by server as a failure to client
[HIVE-18928] - HS2: Perflogger has a race condition
[HIVE-19093] - some parts of the Driver runs from the "Background-Pool" in HS2
[HIVE-19217] - Upgrade to Hadoop 3.1.0
[HIVE-19652] - Incorrect predicate pushdown for groupby with grouping sets
[HIVE-19887] - One item in pom.xml still has 3.1.0 causing Metastore releated tables related test failures.
[HIVE-19955] - ACID: Pre-filter the delete event registry using insert delta ranges
[HIVE-20001] - With doas set to true, running select query as hrt_qa user on external table fails due to permission denied to read /warehouse/tablespace/managed directory.
[HIVE-20304] - When hive.optimize.skewjoin and hive.auto.convert.join are both set to true, and the execution engine is mr, same stage may launch twice due to the wrong generated plan
[HIVE-20435] - Failed Dynamic Partition Insert into insert only table may loose transaction metadata
[HIVE-20470] - Runtime of QueryInfo should be initialized
[HIVE-20520] - length(CHAR) doesn't consider trailing space
[HIVE-20870] - Metastore cache is accessed for transaction table stats
[HIVE-20901] - running compactor when there is nothing to do produces duplicate data
[HIVE-21086] - Fetch task and MR job give back different results when multiple partitions point to the same location
[HIVE-21106] - Potential NEP in VectorizedOrcAcidRowBatchReader.ColumnizedDeleteEventRegistry
[HIVE-21259] - HiveMetaStoreCilent.getNextNotification throws exception when no new events found
[HIVE-21450] - Buffer Reader is not closed during executeInitSql
[HIVE-21513] - ACID: Running merge concurrently with minor compaction causes a later select * to throw exception
[HIVE-21710] - Minor compaction writes delete records in unbucketed tables multiple times when we have multiple files <bucket_N>
[HIVE-21897] - Setting serde / serde properties for partitions
[HIVE-22056] - Beeline started with -f <script-file>, exit code is 0 even if unable to connect to HS2
[HIVE-22317] - Beeline-site parser does not handle the variable substitution correctly
[HIVE-22325] - variable expansion doesn't work in beeline-site.xml
[HIVE-22368] - Hive JDBC Storage Handler: some mysql data type can not be cast to hive data type
[HIVE-22670] - ArrayIndexOutOfBoundsException when vectorized reader is used for reading a parquet file
[HIVE-22758] - Create database with permission error when doas set to true
[HIVE-22806] - Missing materialized view rewrite in case the filter is further narrowed
[HIVE-22956] - Fix checking if a table is used by a materialized view before dropping
[HIVE-22960] - Approximate TopN Key Operator
[HIVE-23056] - LLAP registry getAll doesn't filter compute groups
[HIVE-23500] - [Kubernetes] Use Extend NodeId for LLAP registration
[HIVE-23583] - Upgrade to ant 1.10.9 due to CVEs
[HIVE-23968] - CTAS with TBLPROPERTIES ('transactional'='false') does not entertain translated table location
[HIVE-24070] - ObjectStore.cleanWriteNotificationEvents OutOfMemory on large number of pending events
[HIVE-24086] - CTAS with HMS translation enabled returns empty results.
[HIVE-24122] - When CBO is enable, CAST(STR as Bigint)IS NOT NULL result is wrong
[HIVE-24595] - Vectorization causing incorrect results for scalar subquery
[HIVE-24904] - CVE-2019-10172,CVE-2019-10202 vulnerabilities in jackson-mapper-asl-1.9.13.jar
[HIVE-25298] - LAG function get java.lang.ClassCastException: org.apache.hadoop.hive.common.type.HiveDecimal cannot be cast to org.apache.hadoop.hive.serde2.io.HiveDecimalWritable
[HIVE-25416] - Hive metastore memory leak because datanucleus-api-jdo bug
[HIVE-25421] - Fallback from vectorization when reading Iceberg's time columns from ORC files
[HIVE-25437] - org.pentaho:pentaho-aggdesigner-algorithm has been sunset.
[HIVE-25492] - Major query-based compaction is skipped if partition is empty
[HIVE-25518] - CompactionTxHandler NPE if no CompactionInfo
[HIVE-25621] - Alter table partition compact/concatenate commands should send HivePrivilegeObjects for Authz
[HIVE-25758] - OOM due to recursive application of CBO rules
[HIVE-25798] - Update pom.xml
[HIVE-25802] - Log4j2 Vulnerability in Hive Storage API
[HIVE-25811] - Fix TestMiniLlapLocalCliDriver#testCliDriver[sysdb] caused by the HIVE-25737
[HIVE-25813] - CREATE TABLE x LIKE storagehandler-based-source fails
[HIVE-25848] - Empty result for structs in point lookup optimization with vectorization on
[HIVE-25853] - Security Vulnerability CVE-2021-44832 log4j2 need upgrade to 2.17.1
[HIVE-25879] - MetaStoreDirectSql test query should not query the whole DBS table
[HIVE-25887] - Add external_table_concatenate.q to testconfiguration.properties
[HIVE-25907] - IOW Directory queries fails to write data to final path when query result cache is enabled
[HIVE-25976] - Cleaner may remove files being accessed from a fetch-task-converted reader
[HIVE-25993] - Query-based compaction doesn't work when partition column type is boolean
[HIVE-26055] - Fix the HivePrivilegesObjects for Alter table rename command
[HIVE-26060] - Invalidate acid table directory cache on drop table
[HIVE-26061] - Do not add 'from deserializer' comment upon alter commands for Iceberg tables
[HIVE-26074] - PTF Vectorization: BoundaryScanner for varchar
[HIVE-26096] - Select on single column MultiDelimitSerDe table throws AIOBE
[HIVE-26110] - Bulk insert into partitioned table creates lots of files in iceberg
[HIVE-26114] - Fix jdbc connection hiveserver2 using dfs command with prefix space will cause exception
[HIVE-26115] - LLAP cache utilization for Iceberg Parquet files
[HIVE-26116] - Fix handling of compaction requests originating from aborted dynamic partition queries in Initiator
[HIVE-26127] - INSERT OVERWRITE throws FileNotFound when destination partition is deleted
[HIVE-26131] - Incorrect OutputFormat when describing jdbc connector table
[HIVE-26135] - Invalid Anti join conversion may cause missing results
[HIVE-26139] - URL Encoding from HIVE-26015 was a bit too aggressive
[HIVE-26141] - Fix vector_ptf_part_simple_all_datatypes source file
[HIVE-26147] - OrcRawRecordMerger throws NPE when hive.acid.key.index is missing for an acid file
[HIVE-26158] - TRANSLATED_TO_EXTERNAL partition tables cannot query partition data after rename table
[HIVE-26161] - Use Hive's ORC dependency version when producing file footer for Iceberg
[HIVE-26178] - Multiple version of woodstox jars found in spark class path
[HIVE-26180] - Change MySQLConnectorProvider driver from mariadb to mysql
[HIVE-26184] - COLLECT_SET with GROUP BY is very slow when some keys are highly skewed
[HIVE-26189] - Iceberg metadata query throws exceptions after partition evolution
[HIVE-26192] - JDBC data connector queries occur exception at cbo stage
[HIVE-26193] - Fix Iceberg partitioned tables null bucket handling
[HIVE-26197] - Hive - Upgrade Ant to 1.10.11 due to CVE-2021-36373 and CVE-2021-36374
[HIVE-26205] - Remove the incorrect org.slf4j dependency in kafka-handler
[HIVE-26210] - Fix tests for Cleaner failed attempt threshold
[HIVE-26231] - Generate insert notification events when dynamic partition insert is done on existing partitions
[HIVE-26233] - Problems reading back PARQUET timestamps above 10000 years
[HIVE-26235] - OR Condition on binary column is returning empty result
[HIVE-26249] - slow data processs
[HIVE-26264] - Iceberg integration: Fetch virtual columns on demand
[HIVE-26266] - Column information is not present in lineage for CTAS when custom location/translated location is used
[HIVE-26267] - Addendum to HIVE-26107: perpared statement is not working on Postgres
[HIVE-26268] - Upgrade snappy-java to 1.1.8.4
[HIVE-26270] - Wrong timestamps when reading Hive 3.1.x Parquet files with vectorized reader
[HIVE-26274] - No vectorization if query has upper case window function
[HIVE-26277] - NPEs and rounding issues in ColumnStatsAggregator classes
[HIVE-26288] - NPE in CompactionTxnHandler.markFailed()
[HIVE-26296] - RuntimeException when executing EXPLAIN CBO JOINCOST on query with JDBC tables
[HIVE-26298] - Selecting complex types on migrated iceberg table does not work
[HIVE-26300] - Upgrade Jackson databind version to 2.12.6.1+ to avoid CVE-2020-36518
[HIVE-26320] - Incorrect results for IN UDF on Parquet column of CHAR/VARCHAR type
[HIVE-26340] - Vectorized PTF operator fails if query has upper case window function
[HIVE-26349] - TestOperatorCmp/TestReOptimization fail silently due to incompatible configuration
[HIVE-26350] - IndexOutOfBoundsException when generating splits for external JDBC table with partition columns
[HIVE-26352] - Tez queue access check fails with GSS Exception on Compaction
[HIVE-26355] - Column compare should be case insensitive for name
[HIVE-26358] - Querying metadata tables does not work for Iceberg tables using HADOOP_TABLE
[HIVE-26366] - PTF Vectorization: BoundaryScanner for boolean
[HIVE-26371] - Constant propagation does not evaluate constraint expressions at merge when CBO is enabled
[HIVE-26372] - QTests depend on mysql docker image are fail
[HIVE-26373] - ClassCastException when reading timestamps from HBase table with Avro data
[HIVE-26374] - Query based compaction fails for tables with CDT and columns with Reserved Keywords
[HIVE-26375] - Invalid materialized view after rebuild if source table was compacted
[HIVE-26380] - Fix NPE when reading a struct field with null value from iceberg table
[HIVE-26388] - ClassCastException when there is non string type column in source table of CTAS query
[HIVE-26390] - TxnHandler.getTxnDbsUpdated() SQL syntax error on Postgres.
[HIVE-26396] - The trunc function has a problem with precision interception and the result has many 0
[HIVE-26403] - Iceberg CTAS fails if source table schema contains decimal column
[HIVE-26404] - HMS memory leak when compaction cleaner fails to remove obsolete files
[HIVE-26408] - Vectorization: Fix deallocation of scratch columns, don't reuse a child ConstantVectorExpression as an output
[HIVE-26409] - Assign NO_TXN operation type for table in global locks for scheduled queries
[HIVE-26410] - Reading nested types within maps in Parquet Iceberg is not supported with vectorization
[HIVE-26421] - HmsClient atler_table_req() is called twice when running an alter operation on iceberg table
[HIVE-26423] - Make commons-pool2 an explicit dependency (and upgrade to 2.11.1)
[HIVE-26426] - Avoid StringIndexOutOfBoundsException in canCBOHandleAst() method.
[HIVE-26438] - Remove unnecessary optimization in canHandleQbForCbo() method
[HIVE-26440] - Duplicate hive-standalone-metastore-server dependency in QFile module
[HIVE-26446] - HiveProtoLoggingHook fails to populate TablesWritten field for partitioned tables.
[HIVE-26447] - Vectorization: wrong results when filter on repeating map key orc table
[HIVE-26466] - NullPointerException on HiveQueryLifeTimeHook:checkAndRollbackCTAS
[HIVE-26468] - Cleanup dependency to FlatBuffers
[HIVE-26476] - Iceberg: map "ORCFILE" to "ORC" while creating an iceberg table
[HIVE-26477] - Iceberg: `CREATE TABLE LIKE STORED BY ICEBERG` failing with NullPointerException
[HIVE-26481] - Cleaner fails with FileNotFoundException
[HIVE-26483] - Use DDL_NO_LOCK when running iceberg CTAS query
[HIVE-26495] - MSCK repair perf issue HMSChecker ThreadPool is blocked at fs.listStatus
[HIVE-26496] - FetchOperator scans delete_delta folders multiple times causing slowness
[HIVE-26504] - User is not able to drop table
[HIVE-26506] - HiveIcebergVectorizedRecordReader doesn't set the selected array
[HIVE-26507] - Do not allow hive to iceberg migration if source table contains CHAR or VARCHAR columns
[HIVE-26510] - Minor compaction creates empty delta files when no prior delta file exists
[HIVE-26511] - Fix NoClassDefFoundError in HMS for HBaseConfiguration
[HIVE-26550] - Const struct equality operator throws error when CBO is off and vectorization is on
[HIVE-26556] - Iceberg: Properties set in HiveIcebergSerde are not propagated to jobconf
[HIVE-26568] - Upgrade Log4j2 to 2.18.0 due to CVEs
[HIVE-26573] - ClassCastException error on window function
[HIVE-26576] - Alter table calls on Iceberg tables can inadvertently change metadata_location
[HIVE-26583] - Ensure iceberg-catalog tests are executed in ptest
[HIVE-26584] - compressed_skip_header_footer_aggr.q is flaky
[HIVE-26608] - Iceberg: Allow parquet write properties to iceberg via session conf and Table Properties
[HIVE-26612] - INT64 Parquet timestamps cannot be read into BIGINT Hive type
[HIVE-26614] - Fix adding custom jars in Job Classpath
[HIVE-26629] - Misleading error message with hive.metastore.limit.partition.request
[HIVE-26633] - Make thrift max message size configurable
[HIVE-26639] - ConstantVectorExpression and ExplainTask shouldn't rely on default charset
[HIVE-26643] - HiveUnionPullUpConstantsRule produces an invalid plan when pulling up constants for nullable fields
[HIVE-26652] - HiveSortPullUpConstantsRule produces an invalid plan when pulling up constants for nullable fields
[HIVE-26657] - [Iceberg] Filter out the metadata.json file when migrating
[HIVE-26723] - JDBC - Configurable canonical name checking for Kerberos
New Feature
[HIVE-25230] - add position and occurrence to instr()
[HIVE-25826] - Support table defaults at each database level
[HIVE-25886] - issue
[HIVE-26015] - CREATE HBase table fails when SERDEPROPERTIES contain special characters
[HIVE-26228] - Implement Iceberg table rollback feature
[HIVE-26251] - Disable vectorization when querying iceberg metadata tables
[HIVE-26395] - Support CREATE TABLE LIKE FILE for PARQUET
[HIVE-26425] - Skip SSL cert verification for downloading JWKS in HS2
[HIVE-26443] - Add priority queueing to compaction
[HIVE-26524] - Use Calcite to remove sections of a query plan known never produces rows
[HIVE-26551] - Support CREATE TABLE LIKE FILE for ORC
Improvement
[HIVE-13288] - Confusing exception message in DagUtils.localizeResource
[HIVE-13353] - SHOW COMPACTIONS should support filtering options
[HIVE-14514] - OrcRecordUpdater should clone writerOptions when creating delete event writers
[HIVE-18772] - Make Acid Cleaner use MIN_HISTORY_LEVEL
[HIVE-20473] - Optimization for materialized views
[HIVE-20800] - Use "posix" for property tarLongFileMode for maven-assembly-plugin
[HIVE-20890] - ACID: Allow whole table ReadLocks to skip all partition locks
[HIVE-21323] - LEFT OUTER JOIN does not generate transitive IS NOT NULL filter on right side
[HIVE-22193] - Graceful Shutdown HiveServer2
[HIVE-22316] - Avoid hostname resolution in LlapInputFormat
[HIVE-22692] - Use only fixDecimalDataTypePhysicalVariations when vectorizing TopNKey operator
[HIVE-23270] - Optimize isValidTxnListState to reduce the numbers of HMS calls
[HIVE-23391] - Non blocking ALTER TABLE add column
[HIVE-24483] - aarch64 Support: Bump protobuf version to 3.21.4
[HIVE-24484] - Upgrade Hadoop to 3.3.1 And Tez to 0.10.2
[HIVE-24639] - Raises SemanticException other than ClassCastException when filter has non-boolean expressions
[HIVE-25055] - Improve the exception handling in HMSHandler
[HIVE-25444] - Make tables based on storage handlers authorization (HIVE-24705) configurable
[HIVE-25635] - Upgrade Thrift to 0.16.0
[HIVE-25733] - Add check-spelling CI action
[HIVE-25827] - Parquet file footer is read multiple times, when multiple splits are created in same file
[HIVE-25872] - Skip tracking of alterDatabase events for replication specific properties.
[HIVE-25980] - Reduce fs calls in HiveMetaStoreChecker.checkTable
[HIVE-26020] - Set dependency scope for json-path, commons-compiler and janino to runtime
[HIVE-26021] - Change integration tests under DBInstallBase to CheckIn tests
[HIVE-26026] - Use the new "REFUSED" compaction state where it makes sense
[HIVE-26058] - Choose meaningful names for the Metastore pool threads
[HIVE-26059] - Eventually clean compactions in "refused" state from compaction history
[HIVE-26107] - Worker shouldn't inject duplicate entries in `ready for cleaning` state into the compaction queue
[HIVE-26109] - Upgrade groovy to 2.4.21 due to CVE
[HIVE-26119] - Remove unnecessary Exceptions from DDLPlanUtils
[HIVE-26122] - Factorize out common docker code between DatabaseRule and AbstractExternalDB
[HIVE-26133] - Insert overwrite on Iceberg tables can result in duplicate entries after partition evolution
[HIVE-26137] - Optimized transfer of Iceberg residual expressions from AM to execution
[HIVE-26146] - Handle missing hive.acid.key.index in the fixacidkeyindex utility
[HIVE-26157] - Change Iceberg storage handler authz URI to metadata location
[HIVE-26167] - QueryStateMap in SessionState is not maintained correctly
[HIVE-26169] - Set non-vectorized mode as default when accessing iceberg tables in avro fileformat
[HIVE-26170] - Code cleanup in jdbc dataconnector
[HIVE-26172] - Upgrade ant to 1.10.12
[HIVE-26173] - Upgrade derby to 10.14.2.0
[HIVE-26182] - Some improvements to make DPP more debuggable
[HIVE-26190] - Implement create iceberg table with metadata location
[HIVE-26196] - Integrate Sonar analysis for the master branch and PRs
[HIVE-26202] - Refactor Iceberg Writers
[HIVE-26203] - Implement alter iceberg table metadata location
[HIVE-26215] - Expose the MIN_HISTORY_LEVEL table through Hive sys database
[HIVE-26217] - Make CTAS use Direct Insert Semantics
[HIVE-26226] - Exclude jdk.tools dep from hive-metastore in upgrade-acid
[HIVE-26230] - Option to URL encode special chars in hbase.column.mapping that are valid HBase column family chars
[HIVE-26238] - Decouple sort filter predicates optimization from digest normalization in CBO
[HIVE-26242] - Compaction heartbeater improvements
[HIVE-26253] - upgrade postgresql to 42.4.1 due to security issues
[HIVE-26258] - Provide an option for enable locking of external tables
[HIVE-26260] - Use `Reader.getSchema` instead of deprecated `Reader.getTypes`
[HIVE-26261] - Fix some issues with Spark engine removal
[HIVE-26265] - REPL DUMP should filter out OpenXacts and unneeded CommitXact/Abort.
[HIVE-26272] - Inline util code that is used from log4j jar
[HIVE-26280] - Copy more data into COMPLETED_COMPACTIONS for better supportability
[HIVE-26282] - Improve iceberg CTAS error message for unsupported types
[HIVE-26294] - Allow substr to take bigint as parameters
[HIVE-26307] - Avoid FS init in FileIO::newInputFile in vectorized Iceberg reads
[HIVE-26321] - Upgrade commons-io to 2.11.0
[HIVE-26322] - Upgrade gson to 2.9.0
[HIVE-26334] - Remove misleading bucketing info from DESCRIBE FORMATTED output for Iceberg tables
[HIVE-26354] - Support expiring snapshots on iceberg table
[HIVE-26367] - Upgrade to tez 0.10.2
[HIVE-26378] - Improve error message for masking over complex data types
[HIVE-26381] - Fix link in Javadoc of MetaStoreUtils::getCurrentClassPaths
[HIVE-26385] - Iceberg integration: Implement merge into iceberg table
[HIVE-26397] - Honour Iceberg sort orders when writing a table
[HIVE-26402] - HiveSchemaTool does not honor metastore-site.xml
[HIVE-26414] - Aborted/Cancelled CTAS operations must initiate cleanup of uncommitted data
[HIVE-26417] - Iceberg integration: disable update and merge iceberg table when split update is off
[HIVE-26428] - Limit usage of LLAP BPWrapper to threads of IO threadpools
[HIVE-26429] - Set default value of hive.txn.xlock.ctas to true and update lineage info for CTAS queries.
[HIVE-26439] - Skip collecting Kafka delegation tokens if PLAINTEXT is set
[HIVE-26444] - Use tez.local.mode.without.network for unit tests
[HIVE-26450] - Use a more lightweight Oracle image
[HIVE-26460] - Upgrade Iceberg dependency to 0.14.0
[HIVE-26471] - New metric for Compaction pooling
[HIVE-26474] - Enable converting insert-only tables to full ACID tables
[HIVE-26478] - Explicitly set Content-Type in QueryProfileServlet
[HIVE-26479] - Add ability to set parameters for query-based compaction
[HIVE-26480] - Improve SYS.COMPACTIONS and SYS.TRANSACTIONS views
[HIVE-26500] - Improve TestHiveMetastore
[HIVE-26502] - Improve LDAP auth to support include generic user filters
[HIVE-26519] - Iceberg: Add support for CTLT queries
[HIVE-26521] - Iceberg: Raise exception when running delete/update statements on V1 tables
[HIVE-26529] - Iceberg: Fix VectorizedSupport support for DECIMAL_64 in HiveIcebergInputFormat.
[HIVE-26532] - Remove logger from critical path in VectorMapJoinInnerLongOperator::processBatch
[HIVE-26535] - Iceberg: Support adding parquet compression type via Table properties
[HIVE-26536] - Enable 'hive.acid.truncate.usebase' by default
[HIVE-26539] - Kryo deserialization improvements
[HIVE-26543] - Improve TxnHandler, TxnUtils, CompactionTxnHandler logging
[HIVE-26554] - Upgrade Iceberg dependency to 0.14.1
[HIVE-26564] - Separate query live operation log and historical operation log
[HIVE-26567] - Hive Security - Upgrade Apache Log4j to 2.18.0 due to critical CVEs
[HIVE-26596] - Hive should enforce merge-on-read delete mode is set for Iceberg V2 tables
[HIVE-26603] - Initiator does not initiate minor compaction for insert-only tables
[HIVE-26613] - Upgrade jettison to 1.5.1 to fix CVE-2022-40149
[HIVE-26617] - Remove unused configurations
[HIVE-26620] - Remove unused imports for ThriftBinaryCLIService class
[HIVE-26623] - Maintain same columns in COMPACTIONS view present in SYS and INFORMATION_SCHEMA
[HIVE-26635] - Maintain update and merge mode on Iceberg V2 tables
[HIVE-26638] - Replace in-house CBO reduce expressions rules with Calcite's built-in classes
[HIVE-26642] - Replace HiveFilterMergeRule with Calcite's built-in implementation
[HIVE-26647] - Implement deserialization API for commit compaction event
[HIVE-26651] - MultiDelimitSerDe shouldn't rely on default charset when returning the deserialized string
[HIVE-26670] - Track every single HTTP request between beeline and hs2
Test
[HIVE-19855] - TestStatsUpdaterThread.testQueueingWithThreads fails often
[HIVE-25423] - Add new test driver to automatically launch and load external database
[HIVE-26044] - Remove hardcoded version references from the tests
[HIVE-26200] - Add tests for Iceberg DELETE statements for every supported type
[HIVE-26461] - Add CI build check for macOS
[HIVE-26494] - Fix flaky test TestJdbcWithMiniHS2 testHttpRetryOnServerIdleTimeout
[HIVE-26619] - Sonar analysis is not run for the master branch
Wish
[HIVE-19624] - the method closeSession of SessionManager has a synchronized, is it able to be removed?
Task
[HIVE-19828] - Rename UDFCurrentDB to GenericUDFCurrentDatabase
[HIVE-21588] - Remove HBase dependency from hive-metastore
[HIVE-22030] - Bumping jackson version to 2.9.9 and 2.9.9.3 (jackson-databind)
[HIVE-23744] - Reduce query startup latency
[HIVE-26019] - Upgrade com.jayway.jsonpath from 2.4.0 to 2.7.0
[HIVE-26023] - Non blocking REPLACE, RENAME COLUMNS
[HIVE-26032] - Upgrade cron-utils to 9.1.6
[HIVE-26051] - Non blocking ALTER MATERIALIZED VIEW REWRITE
[HIVE-26052] - Non blocking DROP MATERIALIZED VIEW implementation
[HIVE-26053] - Non blocking DROP VIEW
[HIVE-26062] - Make sure that running 4.0.0-alpha-2 above 4.0.0-alpha-1 HMS DB reports the correct error
[HIVE-26064] - For Iceberg external table do not set external.table.purge=true by default
[HIVE-26067] - Remove core directory from src
[HIVE-26068] - Add README with build instructions to the src tarball
[HIVE-26069] - Remove unnecessary items from the .gitignore
[HIVE-26070] - Remove the generated files from the source tarball
[HIVE-26076] - Non blocking ADD PARTITION if not exists
[HIVE-26077] - Implement CTAS for Iceberg tables with partition spec
[HIVE-26078] - Upgrade gson to 2.8.9
[HIVE-26080] - Upgrade accumulo-core to 1.10.1
[HIVE-26081] - Upgrade ant to 1.10.9
[HIVE-26092] - Fix javadoc errors for the 4.0.0 release
[HIVE-26093] - Deduplicate org.apache.hadoop.hive.metastore.annotation package-info.java
[HIVE-26099] - Move patched-iceberg packages to org.apache.hive group
[HIVE-26100] - Preparing for 4.0.0-alpha-2 development
[HIVE-26101] - Port Iceberg Hive fix - Hive: Avoid recursive listing in HiveCatalog#renameTable
[HIVE-26103] - Port Iceberg fixes to the iceberg module
[HIVE-26106] - Non blocking ADD, DROP constraint
[HIVE-26121] - Hive transaction rollback should be thread-safe
[HIVE-26129] - Non blocking DROP CONNECTOR
[HIVE-26136] - Implement UPDATE statements for Iceberg tables
[HIVE-26149] - Non blocking DROP DATABASE implementation
[HIVE-26165] - Remove READ locks for ACID tables
[HIVE-26252] - Missing locks in case of MERGE with multiple branches
[HIVE-26289] - Remove useless try catch in DataWritableReadSupport#getWriterDateProleptic
[HIVE-26290] - Remove useless calls to DateTimeFormatter#withZone without assignment
[HIVE-26293] - Migrate remaining exclusive DDL operations to EXCL_WRITE lock & bug fixes
[HIVE-26309] - Remove Log4jConfig junit extension in favor LoggerContextSource
[HIVE-26310] - Remove unused junit runners from test-utils module
[HIVE-26324] - Add "one-row-table" constraints on NOTIFICATION_SEQUENCE table
[HIVE-26331] - Use maven-surefire-plugin version consistently in standalone-metastore modules
[HIVE-26412] - Create interface to fetch available slots during split calculation
[HIVE-26416] - AcidUtils$MetaDataFile.isRawFormatFile() throws InvalidProtocolBufferException for non-ORC file
[HIVE-26433] - StorageHandler authorization doesn't work with "METASTORE" db type
[HIVE-26458] - Add explicit dependency to commons-dbcp2 in hive-exec module
[HIVE-26487] - Auto-compaction may hang or skip some of the eligible items
[HIVE-26566] - Upgrade H2 database version to 2.1.214
[HIVE-26579] - Prepare for Hadoop and Zookeeper switching to Reload4j
[HIVE-26594] - Upgrade netty to 4.1.77 due to CVE-2022-24823
[HIVE-26622] - upgrade aws-java-sdk to 1.12.132 version
[HIVE-26626] - Cut dependencies between HiveXxPullUpConstantsRule and HiveReduceExpressionsRule
[HIVE-26627] - Remove HiveRelBuilder.aggregateCall override and refactor callers to use existing public methods
软件描述
Hive是基于hadoop的一个数据仓库工具,可以将结构化数据文件映射为一张数据库表,并提供类SQL查询功能.
评论