Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id AE322200D41 for ; Tue, 17 Oct 2017 14:41:08 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id AC6A11609DE; Tue, 17 Oct 2017 12:41:08 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 031131609EB for ; Tue, 17 Oct 2017 14:41:06 +0200 (CEST) Received: (qmail 99590 invoked by uid 500); 17 Oct 2017 12:41:06 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 99323 invoked by uid 99); 17 Oct 2017 12:41:05 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 17 Oct 2017 12:41:05 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 63C23DF9FD; Tue, 17 Oct 2017 12:41:04 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: aleksey@apache.org To: commits@cassandra.apache.org Date: Tue, 17 Oct 2017 12:41:06 -0000 Message-Id: <310f08737d604caa91ccd22fcd457a93@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [3/6] cassandra git commit: Don't let users drop or generally break tables in system_distributed archived-at: Tue, 17 Oct 2017 12:41:08 -0000 Don't let users drop or generally break tables in system_distributed patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for CASSANDRA-13813 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/9e37967e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9e37967e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9e37967e Branch: refs/heads/trunk Commit: 9e37967e17a1b223df35c1c7cec4dc4adf0b2d91 Parents: d8a18f9 Author: Aleksey Yeschenko Authored: Thu Oct 5 12:05:17 2017 +0100 Committer: Aleksey Yeschenko Committed: Tue Oct 17 13:17:41 2017 +0100 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../org/apache/cassandra/config/Schema.java | 24 ++++++---- .../cql3/statements/AlterKeyspaceStatement.java | 2 +- .../apache/cassandra/db/ColumnFamilyStore.java | 3 +- src/java/org/apache/cassandra/db/Keyspace.java | 4 +- .../org/apache/cassandra/db/ReadCommand.java | 2 +- .../io/sstable/format/SSTableReader.java | 2 +- .../cassandra/schema/LegacySchemaMigrator.java | 2 +- .../apache/cassandra/schema/SchemaKeyspace.java | 6 +-- .../apache/cassandra/service/ClientState.java | 46 +++++++++++--------- .../apache/cassandra/service/StorageProxy.java | 2 +- .../cassandra/service/StorageService.java | 16 +++++-- .../cassandra/thrift/ThriftValidation.java | 2 +- .../cassandra/tools/nodetool/Cleanup.java | 2 +- 14 files changed, 69 insertions(+), 45 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index db3f502..75f2c29 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.16 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813) * Provide a JMX call to sync schema with local storage (CASSANDRA-13954) * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939) * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922) http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/config/Schema.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java index cc24e1e..6d91d8d 100644 --- a/src/java/org/apache/cassandra/config/Schema.java +++ b/src/java/org/apache/cassandra/config/Schema.java @@ -55,12 +55,12 @@ public class Schema public static final Schema instance = new Schema(); /* system keyspace names (the ones with LocalStrategy replication strategy) */ - public static final Set SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(SystemKeyspace.NAME, SchemaKeyspace.NAME); + public static final Set LOCAL_SYSTEM_KEYSPACE_NAMES = + ImmutableSet.of(SystemKeyspace.NAME, SchemaKeyspace.NAME); /* replicate system keyspace names (the ones with a "true" replication strategy) */ - public static final Set REPLICATED_SYSTEM_KEYSPACE_NAMES = ImmutableSet.of(TraceKeyspace.NAME, - AuthKeyspace.NAME, - SystemDistributedKeyspace.NAME); + public static final Set REPLICATED_SYSTEM_KEYSPACE_NAMES = + ImmutableSet.of(TraceKeyspace.NAME, AuthKeyspace.NAME, SystemDistributedKeyspace.NAME); /** * longest permissible KS or CF name. Our main concern is that filename not be more than 255 characters; @@ -112,9 +112,17 @@ public class Schema /** * @return whether or not the keyspace is a really system one (w/ LocalStrategy, unmodifiable, hardcoded) */ - public static boolean isSystemKeyspace(String keyspaceName) + public static boolean isLocalSystemKeyspace(String keyspaceName) { - return SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()); + return LOCAL_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()); + } + + /** + * @return whether or not the keyspace is a replicated system keyspace (trace, auth, sys-ditributed) + */ + public static boolean isReplicatedSystemKeyspace(String keyspaceName) + { + return REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(keyspaceName.toLowerCase()); } /** @@ -336,7 +344,7 @@ public class Schema private Set getNonSystemKeyspacesSet() { - return Sets.difference(keyspaces.keySet(), SYSTEM_KEYSPACE_NAMES); + return Sets.difference(keyspaces.keySet(), LOCAL_SYSTEM_KEYSPACE_NAMES); } /** @@ -374,7 +382,7 @@ public class Schema keyspaces.values() .stream() - .filter(k -> !Schema.isSystemKeyspace(k.name)) + .filter(k -> !Schema.isLocalSystemKeyspace(k.name)) .forEach(builder::add); return builder.build(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java index be85fef..3ca4d72 100644 --- a/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/AlterKeyspaceStatement.java @@ -56,7 +56,7 @@ public class AlterKeyspaceStatement extends SchemaAlteringStatement KeyspaceMetadata ksm = Schema.instance.getKSMetaData(name); if (ksm == null) throw new InvalidRequestException("Unknown keyspace " + name); - if (Schema.isSystemKeyspace(ksm.name)) + if (Schema.isLocalSystemKeyspace(ksm.name)) throw new InvalidRequestException("Cannot alter system keyspace"); attrs.validate(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/db/ColumnFamilyStore.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 0d19856..536681f 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -1651,7 +1651,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean } writeSnapshotManifest(filesJSONArr, snapshotName); - if (!Schema.SYSTEM_KEYSPACE_NAMES.contains(metadata.ksName) && !Schema.REPLICATED_SYSTEM_KEYSPACE_NAMES.contains(metadata.ksName)) + + if (!Schema.isLocalSystemKeyspace(metadata.ksName) && !Schema.isReplicatedSystemKeyspace(metadata.ksName)) writeSnapshotSchema(snapshotName); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/db/Keyspace.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index 7349f42..3d3e037 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -102,7 +102,7 @@ public class Keyspace public static Keyspace open(String keyspaceName) { - assert initialized || Schema.isSystemKeyspace(keyspaceName); + assert initialized || Schema.isLocalSystemKeyspace(keyspaceName); return open(keyspaceName, Schema.instance, true); } @@ -681,7 +681,7 @@ public class Keyspace public static Iterable system() { - return Iterables.transform(Schema.SYSTEM_KEYSPACE_NAMES, keyspaceTransformer); + return Iterables.transform(Schema.LOCAL_SYSTEM_KEYSPACE_NAMES, keyspaceTransformer); } @Override http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/db/ReadCommand.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java index 2d399d8..c93692a 100644 --- a/src/java/org/apache/cassandra/db/ReadCommand.java +++ b/src/java/org/apache/cassandra/db/ReadCommand.java @@ -451,7 +451,7 @@ public abstract class ReadCommand implements ReadQuery private final int failureThreshold = DatabaseDescriptor.getTombstoneFailureThreshold(); private final int warningThreshold = DatabaseDescriptor.getTombstoneWarnThreshold(); - private final boolean respectTombstoneThresholds = !Schema.isSystemKeyspace(ReadCommand.this.metadata().ksName); + private final boolean respectTombstoneThresholds = !Schema.isLocalSystemKeyspace(ReadCommand.this.metadata().ksName); private final boolean enforceStrictLiveness = metadata.enforceStrictLiveness(); private int liveRows = 0; http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java index 25ba212..fb9343f 100644 --- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java @@ -2213,7 +2213,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted keyspaceNames = new ArrayList<>(); query(query).forEach(row -> keyspaceNames.add(row.getString("keyspace_name"))); - keyspaceNames.removeAll(Schema.SYSTEM_KEYSPACE_NAMES); + keyspaceNames.removeAll(Schema.LOCAL_SYSTEM_KEYSPACE_NAMES); Collection keyspaces = new ArrayList<>(); keyspaceNames.forEach(name -> keyspaces.add(readKeyspace(name))); http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/schema/SchemaKeyspace.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java index b6bf415..578f501 100644 --- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java +++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java @@ -280,7 +280,7 @@ public final class SchemaKeyspace for (String schemaTable : ALL) { String query = String.format("DELETE FROM %s.%s USING TIMESTAMP ? WHERE keyspace_name = ?", NAME, schemaTable); - for (String systemKeyspace : Schema.SYSTEM_KEYSPACE_NAMES) + for (String systemKeyspace : Schema.LOCAL_SYSTEM_KEYSPACE_NAMES) executeOnceInternal(query, timestamp, systemKeyspace); } @@ -402,7 +402,7 @@ public final class SchemaKeyspace private static boolean isSystemKeyspaceSchemaPartition(DecoratedKey partitionKey) { - return Schema.isSystemKeyspace(UTF8Type.instance.compose(partitionKey.getKey())); + return Schema.isLocalSystemKeyspace(UTF8Type.instance.compose(partitionKey.getKey())); } /* @@ -880,7 +880,7 @@ public final class SchemaKeyspace public static Keyspaces fetchNonSystemKeyspaces() { - return fetchKeyspacesWithout(Schema.SYSTEM_KEYSPACE_NAMES); + return fetchKeyspacesWithout(Schema.LOCAL_SYSTEM_KEYSPACE_NAMES); } private static Keyspaces fetchKeyspacesWithout(Set excludedKeyspaceNames) http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/service/ClientState.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/ClientState.java b/src/java/org/apache/cassandra/service/ClientState.java index 43002d2..32849bc 100644 --- a/src/java/org/apache/cassandra/service/ClientState.java +++ b/src/java/org/apache/cassandra/service/ClientState.java @@ -40,7 +40,6 @@ import org.apache.cassandra.exceptions.InvalidRequestException; import org.apache.cassandra.exceptions.UnauthorizedException; import org.apache.cassandra.schema.SchemaKeyspace; import org.apache.cassandra.thrift.ThriftValidation; -import org.apache.cassandra.tracing.TraceKeyspace; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.JVMStabilityInspector; import org.apache.cassandra.utils.CassandraVersion; @@ -55,8 +54,7 @@ public class ClientState private static final Set READABLE_SYSTEM_RESOURCES = new HashSet<>(); private static final Set PROTECTED_AUTH_RESOURCES = new HashSet<>(); - private static final Set ALTERABLE_SYSTEM_KEYSPACES = new HashSet<>(); - private static final Set DROPPABLE_SYSTEM_TABLES = new HashSet<>(); + private static final Set DROPPABLE_SYSTEM_AUTH_TABLES = new HashSet<>(); static { // We want these system cfs to be always readable to authenticated users since many tools rely on them @@ -73,14 +71,10 @@ public class ClientState PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getRoleManager().protectedResources()); } - // allow users with sufficient privileges to alter KS level options on AUTH_KS and - // TRACING_KS, and also to drop legacy tables (users, credentials, permissions) from - // AUTH_KS - ALTERABLE_SYSTEM_KEYSPACES.add(AuthKeyspace.NAME); - ALTERABLE_SYSTEM_KEYSPACES.add(TraceKeyspace.NAME); - DROPPABLE_SYSTEM_TABLES.add(DataResource.table(AuthKeyspace.NAME, PasswordAuthenticator.LEGACY_CREDENTIALS_TABLE)); - DROPPABLE_SYSTEM_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraRoleManager.LEGACY_USERS_TABLE)); - DROPPABLE_SYSTEM_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraAuthorizer.USER_PERMISSIONS)); + // allow users with sufficient privileges to drop legacy tables (users, credentials, permissions) from AUTH_KS + DROPPABLE_SYSTEM_AUTH_TABLES.add(DataResource.table(AuthKeyspace.NAME, PasswordAuthenticator.LEGACY_CREDENTIALS_TABLE)); + DROPPABLE_SYSTEM_AUTH_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraRoleManager.LEGACY_USERS_TABLE)); + DROPPABLE_SYSTEM_AUTH_TABLES.add(DataResource.table(AuthKeyspace.NAME, CassandraAuthorizer.USER_PERMISSIONS)); } // Current user for the session @@ -297,15 +291,21 @@ public class ClientState throws UnauthorizedException, InvalidRequestException { validateKeyspace(keyspace); + if (isInternal) return; + validateLogin(); + preventSystemKSSchemaModification(keyspace, resource, perm); + if ((perm == Permission.SELECT) && READABLE_SYSTEM_RESOURCES.contains(resource)) return; + if (PROTECTED_AUTH_RESOURCES.contains(resource)) if ((perm == Permission.CREATE) || (perm == Permission.ALTER) || (perm == Permission.DROP)) throw new UnauthorizedException(String.format("%s schema is protected", resource)); + ensureHasPermission(perm, resource); } @@ -353,21 +353,25 @@ public class ClientState private void preventSystemKSSchemaModification(String keyspace, DataResource resource, Permission perm) throws UnauthorizedException { - // we only care about schema modification. - if (!((perm == Permission.ALTER) || (perm == Permission.DROP) || (perm == Permission.CREATE))) + // we only care about DDL statements + if (perm != Permission.ALTER && perm != Permission.DROP && perm != Permission.CREATE) return; - // prevent system keyspace modification - if (Schema.isSystemKeyspace(keyspace)) + // prevent ALL local system keyspace modification + if (Schema.isLocalSystemKeyspace(keyspace)) throw new UnauthorizedException(keyspace + " keyspace is not user-modifiable."); - // allow users with sufficient privileges to alter KS level options on AUTH_KS and - // TRACING_KS, and also to drop legacy tables (users, credentials, permissions) from - // AUTH_KS - if (ALTERABLE_SYSTEM_KEYSPACES.contains(resource.getKeyspace().toLowerCase()) - && ((perm == Permission.ALTER && !resource.isKeyspaceLevel()) - || (perm == Permission.DROP && !DROPPABLE_SYSTEM_TABLES.contains(resource)))) + if (Schema.isReplicatedSystemKeyspace(keyspace)) { + // allow users with sufficient privileges to alter replication params of replicated system keyspaces + if (perm == Permission.ALTER && resource.isKeyspaceLevel()) + return; + + // allow users with sufficient privileges to drop legacy tables in replicated system keyspaces + if (perm == Permission.DROP && DROPPABLE_SYSTEM_AUTH_TABLES.contains(resource)) + return; + + // prevent all other modifications of replicated system keyspaces throw new UnauthorizedException(String.format("Cannot %s %s", perm, resource)); } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/service/StorageProxy.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 5af2ad0..e380a3f 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -1476,7 +1476,7 @@ public class StorageProxy implements StorageProxyMBean private static boolean systemKeyspaceQuery(List cmds) { for (ReadCommand cmd : cmds) - if (!Schema.isSystemKeyspace(cmd.metadata().ksName)) + if (!Schema.isLocalSystemKeyspace(cmd.metadata().ksName)) return false; return true; } http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/service/StorageService.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java index 743ed24..e5a50dd 100644 --- a/src/java/org/apache/cassandra/service/StorageService.java +++ b/src/java/org/apache/cassandra/service/StorageService.java @@ -1014,6 +1014,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE { try { + /* + * We use timestamp of 0, intentionally, so that varying timestamps wouldn't cause schema mismatches on + * newly added nodes. + * + * Having the initial/default timestamp as 0 also allows users to make and persist changes to replication + * of our replicated system keyspaces. + * + * In case that we need to make incompatible changes to those kesypaces/tables, we'd need to bump the timestamp + * on per-keyspace/per-table basis. So far we've never needed to. + */ MigrationManager.announceNewKeyspace(ksm, 0, false); } catch (AlreadyExistsException e) @@ -2682,7 +2692,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException { - if (Schema.isSystemKeyspace(keyspaceName)) + if (Schema.isLocalSystemKeyspace(keyspaceName)) throw new RuntimeException("Cleanup of the system keyspace is neither necessary nor wise"); CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL; @@ -2926,7 +2936,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE Map snapshotMap = new HashMap<>(); for (Keyspace keyspace : Keyspace.all()) { - if (Schema.isSystemKeyspace(keyspace.getName())) + if (Schema.isLocalSystemKeyspace(keyspace.getName())) continue; for (ColumnFamilyStore cfStore : keyspace.getColumnFamilyStores()) @@ -2952,7 +2962,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE long total = 0; for (Keyspace keyspace : Keyspace.all()) { - if (Schema.isSystemKeyspace(keyspace.getName())) + if (Schema.isLocalSystemKeyspace(keyspace.getName())) continue; for (ColumnFamilyStore cfStore : keyspace.getColumnFamilyStores()) http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/thrift/ThriftValidation.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java index 5e46459..99aed05 100644 --- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java +++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java @@ -636,7 +636,7 @@ public class ThriftValidation public static void validateKeyspaceNotSystem(String modifiedKeyspace) throws org.apache.cassandra.exceptions.InvalidRequestException { - if (Schema.isSystemKeyspace(modifiedKeyspace)) + if (Schema.isLocalSystemKeyspace(modifiedKeyspace)) throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("%s keyspace is not user-modifiable", modifiedKeyspace)); } http://git-wip-us.apache.org/repos/asf/cassandra/blob/9e37967e/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java index 41e9b01..47c65c8 100644 --- a/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java +++ b/src/java/org/apache/cassandra/tools/nodetool/Cleanup.java @@ -47,7 +47,7 @@ public class Cleanup extends NodeToolCmd for (String keyspace : keyspaces) { - if (Schema.isSystemKeyspace(keyspace)) + if (Schema.isLocalSystemKeyspace(keyspace)) continue; try --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org For additional commands, e-mail: commits-help@cassandra.apache.org