sumanth-pasupuleti commented on a change in pull request #1213: URL: https://github.com/apache/cassandra/pull/1213#discussion_r728998340
########## File path: test/distributed/org/apache/cassandra/distributed/test/PartitionDenylistTest.java ########## @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.distributed.test; + +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import com.google.common.util.concurrent.Uninterruptibles; +import org.junit.Assert; +import org.junit.Test; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.service.StorageProxy; +import org.apache.cassandra.service.StorageService; + +import static org.apache.cassandra.distributed.api.Feature.GOSSIP; +import static org.apache.cassandra.distributed.api.Feature.NETWORK; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; + +public class PartitionDenylistTest extends TestBaseImpl +{ + private static final Logger logger = LoggerFactory.getLogger(PartitionDenylistTest.class); + private static final int testReplicationFactor = 3; + + // Create a four node cluster, populate with some denylist entries, stop all + // the nodes, then bring them up one by one, waiting for each node to complete + // startup before starting the next. + // + // On startup each node runs a SELECT * query on the partition denylist table + // to populate the cache. The whole keyspace is unlikely to be available until + // three of the four nodes are started, so the early nodes will go through several + // cycles of failing to retrieve the partition denylist before succeeding. + // + // with({NETWORK,GOSSIP} is currently required for in-JVM dtests to create + // the distributed system tables. + @Test + public void checkStartupWithoutTriggeringUnavailable() throws IOException, InterruptedException, ExecutionException, TimeoutException + { + int nodeCount = 4; + System.setProperty("cassandra.ring_delay_ms", "5000"); // down from 30s default + System.setProperty("cassandra.consistent.rangemovement", "false"); + System.setProperty("cassandra.consistent.simultaneousmoves.allow", "true"); + try (Cluster cluster = Cluster.build(nodeCount) + .withConfig(config -> config + .with(NETWORK) + .with(GOSSIP) + .set("enable_partition_denylist", true) + .set("denylist_initial_load_retry_seconds", 1)) + .createWithoutStarting()) + { + cluster.forEach(i -> { + i.startup(); + i.runOnInstance(PartitionDenylistTest::waitUntilStarted); + }); + + // Do a cluster-wide no unavailables were recorded while the denylist was loaded. + cluster.forEach(i -> i.runOnInstance(PartitionDenylistTest::checkNoUnavailables)); + } + } + + static private void waitUntilStarted() + { + waitUntilStarted(60, TimeUnit.SECONDS); + } + + // To be called inside the instance with runOnInstance + static private void waitUntilStarted(int waitDuration, TimeUnit waitUnits) + { + long deadlineInMillis = currentTimeMillis() + Math.max(1, waitUnits.toMillis(waitDuration)); + while (!StorageService.instance.getOperationMode().equals("NORMAL")) + { + if (currentTimeMillis() >= deadlineInMillis) + { + throw new RuntimeException("Instance did not reach application state NORMAL before timeout"); + } + Uninterruptibles.sleepUninterruptibly(10, TimeUnit.MILLISECONDS); + } + } + + // To be called inside the instance with runOnInstance + static private void checkNoUnavailables() + { + long deadlineInMillis = currentTimeMillis() + TimeUnit.SECONDS.toMillis(30); + + while (currentTimeMillis() < deadlineInMillis && + StorageProxy.instance.getPartitionDenylistLoadSuccesses() == 0) + { + Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS); + } + + Assert.assertTrue("Partition denylist must have loaded before checking unavailables", + StorageProxy.instance.getPartitionDenylistLoadSuccesses() > 0); + Assert.assertEquals("Initial load of partition denylist should not trigger unavailables", + 0L, StorageProxy.rangeMetrics.unavailables.getCount()); + } + + // To be called inside the isntance with runOnInstance, no nodes are started/stopped Review comment: nit: typo instance ########## File path: src/java/org/apache/cassandra/schema/PartitionDenylist.java ########## @@ -0,0 +1,490 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.schema; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.ImmutableSortedSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.github.benmanes.caffeine.cache.CacheLoader; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.LoadingCache; +import org.apache.cassandra.concurrent.ScheduledExecutors; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.db.PartitionPosition; +import org.apache.cassandra.dht.AbstractBounds; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.exceptions.RequestExecutionException; +import org.apache.cassandra.service.StorageService; +import org.apache.cassandra.service.reads.range.RangeCommands; +import org.apache.cassandra.utils.ByteBufferUtil; + +import static org.apache.cassandra.cql3.QueryProcessor.process; +import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; + +/** + * PartitionDenylist uses the system_distributed.partition_denylist table to maintain a list of denylisted partition keys + * for each keyspace/table. + * + * Keys can be entered manually into the partition_denylist table or via the JMX operation StorageProxyMBean.denylistKey + * + * The denylist is stored as one CQL partition per table, and the denylisted keys are column names in that partition. The denylisted + * keys for each table are cached in memory, and reloaded from the partition_denylist table every 24 hours (default) or when the + * StorageProxyMBean.loadPartitionDenylist is called via JMX. + * + * Concurrency of the cache is provided by the concurrency semantics of the guava LoadingCache. All values (DenylistEntry) are + * immutable collections of keys/tokens which are replaced in whole when the cache refreshes from disk. + * + * Notably, in the current design it's possible for a table *cache expiration instigated* reload to end up violating the + * contract on total denylisted keys allowed in the case where it initially loads with a value less than the DBD + * allowable max per table limit due to global constraint enforcement on initial load. Our load and reload function + * simply enforce the *per table* limit without consideration to what that entails at the global key level. While we + * could track the constrained state and count in DenylistEntry, for now the complexity doesn't seem to justify the + * protection against that edge case. The enforcement should take place on a user-instigated full reload as well as + * error messaging about count violations, so this only applies to situations in which someone adds a key and doesn't + * actively tell the cache to fully reload to take that key into consideration, which one could reasonably expect to be + * an antipattern. Review comment: +1 to this callout -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

