Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
1.2.0, 2.1.0, 3.1.3
-
None
-
None
Description
Unable to drop an external partitioned table after renaming it. Getting the following exception,
java.sql.BatchUpdateException: Cannot delete or update a parent row: a foreign key constraint fails ("hive"."PART_COL_STATS", CONSTRAINT "PART_COL_STATS_FK" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS" ("PART_ID")) at com.mysql.jdbc.PreparedStatement.executeBatchSerially(PreparedStatement.java:2024) at com.mysql.jdbc.PreparedStatement.executeBatch(PreparedStatement.java:1449) at com.jolbox.bonecp.StatementHandle.executeBatch(StatementHandle.java:424) at org.datanucleus.store.rdbms.ParamLoggingPreparedStatement.executeBatch(ParamLoggingPreparedStatement.java:366) at org.datanucleus.store.rdbms.SQLController.processConnectionStatement(SQLController.java:676) at org.datanucleus.store.rdbms.SQLController.getStatementForUpdate(SQLController.java:204) at org.datanucleus.store.rdbms.SQLController.getStatementForUpdate(SQLController.java:176) at org.datanucleus.store.rdbms.scostore.JoinMapStore.clearInternal(JoinMapStore.java:900) at org.datanucleus.store.rdbms.scostore.JoinMapStore.clear(JoinMapStore.java:449) at org.datanucleus.store.types.wrappers.backed.Map.clear(Map.java:605) at org.datanucleus.store.rdbms.mapping.java.MapMapping.preDelete(MapMapping.java:252) at org.datanucleus.store.rdbms.request.DeleteRequest.execute(DeleteRequest.java:193) at org.datanucleus.store.rdbms.RDBMSPersistenceHandler.deleteObjectFromTable(RDBMSPersistenceHandler.java:499) at org.datanucleus.store.rdbms.RDBMSPersistenceHandler.deleteObject(RDBMSPersistenceHandler.java:470) at org.datanucleus.state.AbstractStateManager.internalDeletePersistent(AbstractStateManager.java:832) at org.datanucleus.state.StateManagerImpl.deletePersistent(StateManagerImpl.java:4244) at org.datanucleus.ExecutionContextImpl.deleteObjectInternal(ExecutionContextImpl.java:2395) at org.datanucleus.ExecutionContextImpl.deleteObjectWork(ExecutionContextImpl.java:2317) at org.datanucleus.ExecutionContextImpl.deleteObjects(ExecutionContextImpl.java:2209) at org.datanucleus.ExecutionContextThreadedImpl.deleteObjects(ExecutionContextThreadedImpl.java:259) at org.datanucleus.store.query.Query.performDeletePersistentAll(Query.java:2133) at org.datanucleus.store.query.AbstractJavaQuery.performDeletePersistentAll(AbstractJavaQuery.java:114) at org.datanucleus.store.query.Query.deletePersistentAll(Query.java:2085) at org.datanucleus.api.jdo.JDOQuery.deletePersistentInternal(JDOQuery.java:441) at org.datanucleus.api.jdo.JDOQuery.deletePersistentAll(JDOQuery.java:428) at org.apache.hadoop.hive.metastore.ObjectStore.dropPartitionsNoTxn(ObjectStore.java:2421) at org.apache.hadoop.hive.metastore.ObjectStore.dropPartitions(ObjectStore.java:1805) at sun.reflect.GeneratedMethodAccessor78.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.hive.metastore.RawStoreProxy.invoke(RawStoreProxy.java:103) at com.sun.proxy.$Proxy10.dropPartitions(Unknown Source) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.dropPartitionsAndGetLocations(HiveMetaStore.java:1838) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.drop_table_core(HiveMetaStore.java:1673) at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.drop_table_with_environment_context(HiveMetaStore.java:1859) at sun.reflect.GeneratedMethodAccessor110.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invokeInternal(RetryingHMSHandler.java:147) at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:105) at com.sun.proxy.$Proxy12.drop_table_with_environment_context(Unknown Source) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Processor$drop_table_with_environment_context.getResult(ThriftHiveMetastore.java:9523) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Processor$drop_table_with_environment_context.getResult(ThriftHiveMetastore.java:9507) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:551) at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor$1.run(HadoopThriftAuthBridge.java:546) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1866) at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:546) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: com.mysql.jdbc.exceptions.jdbc4.MySQLIntegrityConstraintViolationException: Cannot delete or update a parent row: a foreign key constraint fails ("hive"."PART_COL_STATS", CONSTRAINT "PART_COL_STATS_FK" FOREIGN KEY ("PART_ID") REFERENCES "PARTITIONS" ("PART_ID")) at sun.reflect.GeneratedConstructorAccessor344.newInstance(Unknown Source) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at com.mysql.jdbc.Util.handleNewInstance(Util.java:411) at com.mysql.jdbc.Util.getInstance(Util.java:386) at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:1040) at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3597) at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3529) at com.mysql.jdbc.MysqlIO.sendCommand(MysqlIO.java:1990) at com.mysql.jdbc.MysqlIO.sqlQueryDirect(MysqlIO.java:2151) at com.mysql.jdbc.ConnectionImpl.execSQL(ConnectionImpl.java:2625) at com.mysql.jdbc.PreparedStatement.executeInternal(PreparedStatement.java:2119) at com.mysql.jdbc.PreparedStatement.executeUpdate(PreparedStatement.java:2415) at com.mysql.jdbc.PreparedStatement.executeBatchSerially(PreparedStatement.java:1979) ... 54 more
Renaming was done using the following command,
hive -e "alter table <DB>.<OLD-Table-name> rename to <DB>.<New-Table-name>;
STEPS TO REPRODUCE
===================
1. Create an external table in hive.
2. . Add sample data to the table.
3. Rename the table using the above command.
4. Drop the renamed table.
Attachments
Issue Links
- relates to
-
HIVE-6980 Drop table by using direct sql
- Closed