Description
I notice this problem when trying to issue some GRANT commands where the transactions fail after exhausting the number of retries to commit to the database.
Current load conditions:
- 20 parallel clients each issuing a GRANT command on MariaDB backed Sentry server.
Even with this relatively small load, we came across this issue which could get become a major problem under high load. Looking at the retry logic, we do have static wait before retrying but most probably all the clients are waking up at the same time which keeps hitting the same problem and thus exhausts all the retries.
Instead of a static wait, we can think of implementing an exponential backoff as a way to avoid this.
Caused by: java.lang.RuntimeException: Unknown error for request: TAlterSentryRoleGrantPrivilegeRequest(protocol_version:2, requestorUserName:hive, roleName:gr_role_2017_06_27_16_48_10_0_0_8, privileges:[TSentryPrivilege(privilegeScope:DATABASE, serverName:server1, dbName:gr_db_2017_06_27_16_48_10, tableName:, URI:, action:*, createTime:1498607304833, grantOption:FALSE, columnName:)]), message: The transaction has reached max retry number, Exception thrown when executing query. Server Stacktrace: java.lang.Exception: The transaction has reached max retry number, Exception thrown when executing query at org.apache.sentry.provider.db.service.persistent.TransactionManager.executeTransactionBlocksWithRetry(TransactionManager.java:224) at org.apache.sentry.provider.db.service.persistent.SentryStore.execute(SentryStore.java:3272) at org.apache.sentry.provider.db.service.persistent.SentryStore.alterSentryRoleGrantPrivilege(SentryStore.java:589) at org.apache.sentry.provider.db.service.persistent.SentryStore.alterSentryRoleGrantPrivileges(SentryStore.java:629) at org.apache.sentry.provider.db.service.thrift.SentryPolicyStoreProcessor.alter_sentry_role_grant_privilege(SentryPolicyStoreProcessor.java:256) at org.apache.sentry.provider.db.service.thrift.SentryPolicyService$Processor$alter_sentry_role_grant_privilege.getResult(SentryPolicyService.java:997) at org.apache.sentry.provider.db.service.thrift.SentryPolicyService$Processor$alter_sentry_role_grant_privilege.getResult(SentryPolicyService.java:982) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.sentry.provider.db.service.thrift.SentryProcessorWrapper.process(SentryProcessorWrapper.java:36) at org.apache.thrift.TMultiplexedProcessor.process(TMultiplexedProcessor.java:123) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:286) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) Caused by: javax.jdo.JDOException: Exception thrown when executing query NestedThrowables: java.sql.BatchUpdateException: Duplicate entry '1153' for key 'PRIMARY' at org.datanucleus.api.jdo.NucleusJDOHelper.getJDOExceptionForNucleusException(NucleusJDOHelper.java:596) at org.datanucleus.api.jdo.JDOQuery.execute(JDOQuery.java:252) at org.apache.sentry.provider.db.service.persistent.SentryStore.getRole(SentryStore.java:291) at org.apache.sentry.provider.db.service.persistent.SentryStore.alterSentryRoleGrantPrivilegeCore(SentryStore.java:641) at org.apache.sentry.provider.db.service.persistent.SentryStore.access$500(SentryStore.java:101) at org.apache.sentry.provider.db.service.persistent.SentryStore$11.execute(SentryStore.java:597) at org.apache.sentry.provider.db.service.persistent.TransactionManager.executeTransaction(TransactionManager.java:148) at org.apache.sentry.provider.db.service.persistent.TransactionManager.executeTransactionBlocksWithRetry(TransactionManager.java:214) ... 14 more Caused by: java.sql.BatchUpdateException: Duplicate entry '1153' for key 'PRIMARY' at com.mysql.jdbc.PreparedStatement.executeBatchSerially(PreparedStatement.java:1815) at com.mysql.jdbc.PreparedStatement.executeBatch(PreparedStatement.java:1277) at com.jolbox.bonecp.StatementHandle.executeBatch(StatementHandle.java:469) at org.datanucleus.store.rdbms.ParamLoggingPreparedStatement.executeBatch(ParamLoggingPreparedStatement.java:372) at org.datanucleus.store.rdbms.SQLController.processConnectionStatement(SQLController.java:683) at org.datanucleus.store.rdbms.SQLController.getStatementForQuery(SQLController.java:324) at org.datanucleus.store.rdbms.query.RDBMSQueryUtils.getPreparedStatementForQuery(RDBMSQueryUtils.java:217) at org.datanucleus.store.rdbms.query.JDOQLQuery.performExecute(JDOQLQuery.java:633) at org.datanucleus.store.query.Query.executeQuery(Query.java:1786) at org.datanucleus.store.query.Query.executeWithArray(Query.java:1672) at org.datanucleus.api.jdo.JDOQuery.execute(JDOQuery.java:243) ... 20 more Caused by: com.mysql.jdbc.exceptions.jdbc4.MySQLIntegrityConstraintViolationException: Duplicate entry '1153' for key 'PRIMARY' at sun.reflect.GeneratedConstructorAccessor37.newInstance(Unknown Source) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:526) at com.mysql.jdbc.Util.handleNewInstance(Util.java:377) at com.mysql.jdbc.Util.getInstance(Util.java:360) at com.mysql.jdbc.SQLError.createSQLException(SQLError.java:971) at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3887) at com.mysql.jdbc.MysqlIO.checkErrorPacket(MysqlIO.java:3823) at com.mysql.jdbc.MysqlIO.sendCommand(MysqlIO.java:2435) at com.mysql.jdbc.MysqlIO.sqlQueryDirect(MysqlIO.java:2582) at com.mysql.jdbc.ConnectionImpl.execSQL(ConnectionImpl.java:2530) at com.mysql.jdbc.PreparedStatement.executeInternal(PreparedStatement.java:1907) at com.mysql.jdbc.PreparedStatement.executeUpdate(PreparedStatement.java:2141) at com.mysql.jdbc.PreparedStatement.executeBatchSerially(PreparedStatement.java:1773) ... 30 more at org.apache.sentry.service.thrift.Status.throwIfNotOk(Status.java:109) at org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClientDefaultImpl.grantPrivilege(SentryPolicyServiceClientDefaultImpl.java:388) at org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClientDefaultImpl.grantDatabasePrivilege(SentryPolicyServiceClientDefaultImpl.java:285) ... 28 more
Attachments
Attachments
Issue Links
- relates to
-
SENTRY-1824 SentryStore may serialize transactions that rely on unique key
- Resolved
- links to