[ 
https://issues.apache.org/jira/browse/CASSANDRA-15041?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16868818#comment-16868818
 ] 

Sam Tunnicliffe edited comment on CASSANDRA-15041 at 6/20/19 7:38 PM:
----------------------------------------------------------------------

Thanks for this [~eperott], there are definitely valuable improvements here.

Whilst the new dtests for handling unavailability do what they claim, they’re 
not doing so in the way one *might* expect. First of all, because they use a 
superuser, table/keyspace level permissions will never be read from system 
tables and so won’t be vulnerable to unavailablity. The reason the tests are 
currently passing is that the check that the user has access to the local DC is 
resulting in a failed read by {{CassandraNetworkAuthorizer}}. So the tests are 
definitely still valid as they are, but it might be worth adding some 
commentary just to highlight this and maybe use a non-superuser role that 
*could* exercise the permissions cache too (even though it won't with the way 
the current test are organised). Potentially, adding some logging to the new 
catch blocks in {{CassandraRoleManager}} might be useful, similar to that in 
{{CassandraAuthorizer::authorize}}. {{CassandraNetworkAuthorizer}} is 
especially vulnerable here, due to CASSANDRA-15089, which I will commit as soon 
as I get chance.

Otherwise, this all looks pretty good to me, modulo a typo in 
{{ClientState::canLogin}} (only in 2.2/3.0/3.11).

I’ve been running the tests with the HIRES circle configuration and everything 
looks good - all failures are unrelated and/or fixed already by other tickets.
 
So I just have these nits:

* typo in {{ClientState::canLogin}} (2.2/3.0/3.11)
* maybe add some minimal logging in {{CassandraRoleManager::isSuper/canLogin}} 
or make the exception messages distinguishable
* document the potential sources of the availability failures that the dtests 
are checking and/or use a non-superuser in tests

One thing that I had missed up to now was that when Guava was replaced with 
Caffeine in CASSANDRA-10855, the ability to do async reloading in {{AuthCache}} 
was lost. We should definitely fix that, as having those reads done in a thread 
servicing client reads is not a good idea. I’ve opened CASSANDRA-15177 for 
that. 



was (Author: beobal):
Thanks for this [~eperott], there are definitely valuable improvements here.

Whilst the new dtests for handling unavailability do what they claim, they’re 
not doing so in the way one *might* expect. First of all, because they use a 
superuser, table/keyspace level permissions will never be read from system 
tables and so won’t be vulnerable to unavailablity. The reason the tests are 
currently passing is that the check that the user has access to the local DC is 
resulting in a failed read by {{CassandraNetworkAuthorizer}}. So the tests are 
definitely still valid as they are, but it might be worth adding some 
commentary just to highlight this and maybe use a non-superuser role to 
exercise the permissions cache too. Potentially, adding some logging to the new 
catch blocks in {{CassandraRoleManager}} might be useful, similar to that in 
{{CassandraAuthorizer::authorize}}. {{CassandraNetworkAuthorizer}} is 
especially vulnerable here, due to CASSANDRA-15089, which I will commit as soon 
as I get chance.

Otherwise, this all looks pretty good to me, modulo a typo in 
{{ClientState::canLogin}} (only in 2.2/3.0/3.11).

I’ve been running the tests with the HIRES circle configuration and everything 
looks good - all failures are unrelated and/or fixed already by other tickets.
 
So I just have these nits:

* typo in {{ClientState::canLogin}} (2.2/3.0/3.11)
* maybe add some minimal logging in {{CassandraRoleManager::isSuper/canLogin}} 
or make the exception messages distinguishable
* document the potential sources of the availability failures that the dtests 
are checking and/or use a non-superuser in tests

One thing that I had missed up to now was that when Guava was replaced with 
Caffeine in CASSANDRA-10855, the ability to do async reloading in {{AuthCache}} 
was lost. We should definitely fix that, as having those reads done in a thread 
servicing client reads is not a good idea. I’ve opened CASSANDRA-15177 for 
that. 


> UncheckedExecutionException if authentication/authorization query fails
> -----------------------------------------------------------------------
>
>                 Key: CASSANDRA-15041
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-15041
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Feature/Authorization
>            Reporter: Per Otterström
>            Assignee: Per Otterström
>            Priority: Normal
>             Fix For: 2.2.15, 3.0.19, 3.11.5, 4.0
>
>
> If cache update for permissions/credentials/roles fails with 
> UnavailableException this comes back to client as UncheckedExecutionException.
> Stack trace on server side:
> {noformat}
> ERROR [Native-Transport-Requests-1] 2019-03-04 16:30:51,537 
> ErrorMessage.java:384 - Unexpected exception during request
> com.google.common.util.concurrent.UncheckedExecutionException: 
> com.google.common.util.concurrent.UncheckedExecutionException: 
> java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.UnavailableException: Cannot achieve 
> consistency level QUORUM
>         at 
> com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) 
> ~[guava-18.0.jar:na]
>         at com.google.common.cache.LocalCache.get(LocalCache.java:3937) 
> ~[guava-18.0.jar:na]
>         at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3941) 
> ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4824)
>  ~[guava-18.0.jar:na]
>         at org.apache.cassandra.auth.AuthCache.get(AuthCache.java:97) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.PermissionsCache.getPermissions(PermissionsCache.java:45)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.AuthenticatedUser.getPermissions(AuthenticatedUser.java:104)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.ClientState.authorize(ClientState.java:439) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.ClientState.checkPermissionOnResourceChain(ClientState.java:368)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.ClientState.ensureHasPermission(ClientState.java:345)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.ClientState.hasAccess(ClientState.java:332) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.ClientState.hasColumnFamilyAccess(ClientState.java:310)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.statements.ModificationStatement.checkAccess(ModificationStatement.java:211)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:222)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:532)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:509)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:146)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:566)
>  [apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:410)
>  [apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105)
>  [netty-all-4.0.44.Final.jar:4.0.44.Final]
>         at 
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:357)
>  [netty-all-4.0.44.Final.jar:4.0.44.Final]
>         at 
> io.netty.channel.AbstractChannelHandlerContext.access$600(AbstractChannelHandlerContext.java:35)
>  [netty-all-4.0.44.Final.jar:4.0.44.Final]
>         at 
> io.netty.channel.AbstractChannelHandlerContext$7.run(AbstractChannelHandlerContext.java:348)
>  [netty-all-4.0.44.Final.jar:4.0.44.Final]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_181]
>         at 
> org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:162)
>  [apache-cassandra-3.11.4.jar:3.11.4]
>         at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:114) 
> [apache-cassandra-3.11.4.jar:3.11.4]
>         at java.lang.Thread.run(Thread.java:748) [na:1.8.0_181]
> Caused by: com.google.common.util.concurrent.UncheckedExecutionException: 
> java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.UnavailableException: Cannot achieve 
> consistency level QUORUM
>         at 
> com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2203) 
> ~[guava-18.0.jar:na]
>         at com.google.common.cache.LocalCache.get(LocalCache.java:3937) 
> ~[guava-18.0.jar:na]
>         at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3941) 
> ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4824)
>  ~[guava-18.0.jar:na]
>         at org.apache.cassandra.auth.AuthCache.get(AuthCache.java:97) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at org.apache.cassandra.auth.RolesCache.getRoles(RolesCache.java:44) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at org.apache.cassandra.auth.Roles.hasSuperuserStatus(Roles.java:51) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.AuthenticatedUser.isSuper(AuthenticatedUser.java:71)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraAuthorizer.authorize(CassandraAuthorizer.java:81)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.PermissionsCache.lambda$new$0(PermissionsCache.java:37)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at org.apache.cassandra.auth.AuthCache$1.load(AuthCache.java:172) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527)
>  ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) 
> ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282)
>  ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) 
> ~[guava-18.0.jar:na]
>         ... 26 common frames omitted
> Caused by: java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.UnavailableException: Cannot achieve 
> consistency level QUORUM
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:518)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRoles(CassandraRoleManager.java:283)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.RolesCache.lambda$new$0(RolesCache.java:36) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at org.apache.cassandra.auth.AuthCache$1.load(AuthCache.java:172) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3527)
>  ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2319) 
> ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2282)
>  ~[guava-18.0.jar:na]
>         at 
> com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2197) 
> ~[guava-18.0.jar:na]
>         ... 40 common frames omitted
> Caused by: org.apache.cassandra.exceptions.UnavailableException: Cannot 
> achieve consistency level QUORUM
>         at 
> org.apache.cassandra.db.ConsistencyLevel.assureSufficientLiveNodes(ConsistencyLevel.java:334)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.AbstractReadExecutor.getReadExecutor(AbstractReadExecutor.java:162)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.<init>(StorageProxy.java:1766)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1728) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1671) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1586) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.db.SinglePartitionReadCommand$Group.execute(SinglePartitionReadCommand.java:1209)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:315)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:285)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRoleFromTable(CassandraRoleManager.java:526)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:508)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         ... 47 common frames omitted
> {noformat}
> Also, if {{x_validity_in_ms}} > {{x_update_interval_in_ms}}, then the 
> background update thread will fail in a similar way:
> {noformat}
> ERROR [PermissionsCacheRefresh:1] 2019-03-04 16:30:43,541 
> CassandraDaemon.java:228 - Exception in thread 
> Thread[PermissionsCacheRefresh:1,5,main]
> java.lang.RuntimeException: 
> org.apache.cassandra.exceptions.UnavailableException: Cannot achieve 
> consistency level QUORUM
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:518)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.isSuper(CassandraRoleManager.java:307)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at org.apache.cassandra.auth.Roles.hasSuperuserStatus(Roles.java:52) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.AuthenticatedUser.isSuper(AuthenticatedUser.java:71)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraAuthorizer.authorize(CassandraAuthorizer.java:81)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.PermissionsCache.lambda$new$0(PermissionsCache.java:37)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.AuthCache$1.lambda$reload$0(AuthCache.java:180) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> ~[na:1.8.0_181]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  ~[na:1.8.0_181]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  [na:1.8.0_181]
>         at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
>  [apache-cassandra-3.11.4.jar:3.11.4]
>         at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_181]
> Caused by: org.apache.cassandra.exceptions.UnavailableException: Cannot 
> achieve consistency level QUORUM
>         at 
> org.apache.cassandra.db.ConsistencyLevel.assureSufficientLiveNodes(ConsistencyLevel.java:334)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.AbstractReadExecutor.getReadExecutor(AbstractReadExecutor.java:162)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy$SinglePartitionReadLifecycle.<init>(StorageProxy.java:1766)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy.fetchRows(StorageProxy.java:1728) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy.readRegular(StorageProxy.java:1671) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.service.StorageProxy.read(StorageProxy.java:1586) 
> ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.db.SinglePartitionReadCommand$Group.execute(SinglePartitionReadCommand.java:1209)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:315)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:285)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRoleFromTable(CassandraRoleManager.java:526)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         at 
> org.apache.cassandra.auth.CassandraRoleManager.getRole(CassandraRoleManager.java:508)
>  ~[apache-cassandra-3.11.4.jar:3.11.4]
>         ... 11 common frames omitted
> {noformat}
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to