You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/12/17 00:08:10 UTC
[1/5] cassandra git commit: Isolate schema serializaton code
Repository: cassandra
Updated Branches:
refs/heads/trunk 32ac6af2b -> 3e9d345f0
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/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 999d6b6..a349e64 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -3259,7 +3259,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
private Future<StreamState> streamHints()
{
// StreamPlan will not fail if there are zero files to transfer, so flush anyway (need to get any in-memory hints, as well)
- ColumnFamilyStore hintsCF = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+ ColumnFamilyStore hintsCF = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS);
FBUtilities.waitOnFuture(hintsCF.forceFlush());
// gather all live nodes in the cluster that aren't also leaving
@@ -3292,7 +3292,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
preferred,
SystemKeyspace.NAME,
ranges,
- SystemKeyspace.HINTS_TABLE)
+ SystemKeyspace.HINTS)
.execute();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/thrift/ThriftConversion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
index b04a091..d408767 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.cql3.Operator;
import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.db.ColumnFamilyType;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.db.WriteType;
import org.apache.cassandra.db.composites.CellNameType;
import org.apache.cassandra.db.composites.CellNames;
@@ -330,9 +331,9 @@ public class ThriftConversion
List<Map<String, ByteBuffer>> cols = new ArrayList<>(columnsRes.rows.size());
for (CqlRow row : columnsRes.rows)
cols.add(convertThriftCqlRow(row));
- UntypedResultSet colsRow = UntypedResultSet.create(cols);
+ UntypedResultSet colsRows = UntypedResultSet.create(cols);
- return CFMetaData.fromSchemaNoTriggers(cfRow, colsRow);
+ return LegacySchemaTables.createTableFromTableRowAndColumnRows(cfRow, colsRows);
}
private static Map<String, ByteBuffer> convertThriftCqlRow(CqlRow row)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/BulkLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index 8e7e0e9..a720e12 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -26,12 +26,14 @@ import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import org.apache.commons.cli.*;
+
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.protocol.TProtocol;
import org.apache.thrift.transport.TTransport;
import org.apache.cassandra.auth.IAuthenticator;
import org.apache.cassandra.config.*;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.dht.Range;
@@ -309,7 +311,7 @@ public class BulkLoader
String cfQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = '%s'",
SystemKeyspace.NAME,
- SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE,
+ LegacySchemaTables.COLUMNFAMILIES,
keyspace);
CqlResult cfRes = client.execute_cql3_query(ByteBufferUtil.bytes(cfQuery), Compression.NONE, ConsistencyLevel.ONE);
@@ -319,7 +321,7 @@ public class BulkLoader
String columnFamily = UTF8Type.instance.getString(row.columns.get(1).bufferForName());
String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
SystemKeyspace.NAME,
- SystemKeyspace.SCHEMA_COLUMNS_TABLE,
+ LegacySchemaTables.COLUMNS,
keyspace,
columnFamily);
CqlResult columnsRes = client.execute_cql3_query(ByteBufferUtil.bytes(columnsQuery), Compression.NONE, ConsistencyLevel.ONE);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index fa6b973..64dc2d8 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -26,7 +26,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.cli.*;
import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.composites.CellNameType;
@@ -420,7 +419,7 @@ public class SSTableExport
String[] excludes = cmd.getOptionValues(EXCLUDEKEY_OPTION);
String ssTableFileName = new File(cmd.getArgs()[0]).getAbsolutePath();
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
Descriptor descriptor = Descriptor.fromFilename(ssTableFileName);
// Start by validating keyspace name
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 6f5dc9d..f51610b 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -502,7 +502,7 @@ public class SSTableImport
isSorted = true;
}
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
if (Schema.instance.getNonSystemKeyspaces().size() < 1)
{
String msg = "no non-system keyspaces are defined";
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
index 3954b91..815620e 100644
--- a/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
+++ b/src/java/org/apache/cassandra/tools/SSTableLevelResetter.java
@@ -21,7 +21,6 @@ import java.io.PrintStream;
import java.util.Map;
import java.util.Set;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
@@ -62,7 +61,7 @@ public class SSTableLevelResetter
try
{
// load keyspace descriptions.
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
String keyspaceName = args[1];
String columnfamily = args[2];
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 94bf6e2..b6e2bf8 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -24,7 +24,6 @@ import java.util.*;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.cli.*;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
@@ -53,7 +52,7 @@ public class StandaloneScrubber
try
{
// load keyspace descriptions.
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
if (Schema.instance.getCFMetaData(options.keyspaceName, options.cfName) == null)
throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index 66cb361..90ab48e 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -21,10 +21,10 @@ package org.apache.cassandra.tools;
import java.io.File;
import java.util.*;
+import org.apache.cassandra.config.Schema;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.commons.cli.*;
-import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
@@ -52,7 +52,7 @@ public class StandaloneSplitter
try
{
// load keyspace descriptions.
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
String ksName = null;
String cfName = null;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index 90eca5d..6617b96 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -46,7 +46,7 @@ public class StandaloneUpgrader
try
{
// load keyspace descriptions.
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
if (Schema.instance.getCFMetaData(options.keyspace, options.cf) == null)
throw new IllegalArgumentException(String.format("Unknown keyspace/table %s.%s",
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
index 79373ea..5fed5be 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -24,20 +24,21 @@ import java.util.HashMap;
import java.util.HashSet;
import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.marshal.AsciiType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.compress.*;
import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.thrift.CfDef;
import org.apache.cassandra.thrift.ColumnDef;
import org.apache.cassandra.thrift.IndexType;
import org.apache.cassandra.thrift.ThriftConversion;
import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
import org.junit.BeforeClass;
import org.junit.Test;
@@ -132,6 +133,7 @@ public class CFMetaDataTest
private void checkInverses(CFMetaData cfm) throws Exception
{
DecoratedKey k = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(cfm.ksName));
+ KSMetaData keyspace = Schema.instance.getKSMetaData(cfm.ksName);
// Test thrift conversion
CFMetaData before = cfm;
@@ -139,11 +141,10 @@ public class CFMetaDataTest
assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
// Test schema conversion
- Mutation rm = cfm.toSchema(System.currentTimeMillis());
- ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE));
- ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNS_TABLE));
- UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", new Row(k, serializedCf)).one();
- CFMetaData newCfm = CFMetaData.fromSchemaNoTriggers(result, ColumnDefinition.resultify(new Row(k, serializedCD)));
+ Mutation rm = LegacySchemaTables.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros());
+ ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNFAMILIES));
+ ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNS));
+ CFMetaData newCfm = LegacySchemaTables.createTableFromTablePartitionAndColumnsPartition(new Row(k, serializedCf), new Row(k, serializedCD));
assert cfm.equals(newCfm) : String.format("%n%s%n!=%n%s", cfm, newCfm);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index edfd7b6..8ca7a79 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -56,9 +56,10 @@ public class DatabaseDescriptorTest
@Test
public void testKSMetaDataSerialization() throws ConfigurationException
{
- for (KSMetaData ksm : Schema.instance.getKeyspaceDefinitions())
+ for (String ks : Schema.instance.getNonSystemKeyspaces())
{
// Not testing round-trip on the KsDef via serDe() because maps
+ KSMetaData ksm = Schema.instance.getKSMetaData(ks);
KSMetaData ksmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(ksm));
assertNotNull(ksmDupe);
assertEquals(ksm, ksmDupe);
@@ -70,7 +71,7 @@ public class DatabaseDescriptorTest
public void testTransKsMigration() throws ConfigurationException
{
SchemaLoader.cleanupAndLeaveDirs();
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
assertEquals(0, Schema.instance.getNonSystemKeyspaces().size());
Gossiper.instance.start((int)(System.currentTimeMillis() / 1000));
@@ -91,7 +92,7 @@ public class DatabaseDescriptorTest
assertNull(Schema.instance.getKSMetaData("ks0"));
assertNull(Schema.instance.getKSMetaData("ks1"));
- DatabaseDescriptor.loadSchemas();
+ Schema.instance.loadFromDisk();
assertNotNull(Schema.instance.getKSMetaData("ks0"));
assertNotNull(Schema.instance.getKSMetaData("ks1"));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
deleted file mode 100644
index 6df67bc..0000000
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ /dev/null
@@ -1,564 +0,0 @@
-/**
- * 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.config;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.TimeUUIDType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableDeletingTask;
-import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import static org.apache.cassandra.Util.cellname;
-
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-@RunWith(OrderedJUnit4ClassRunner.class)
-public class DefsTest
-{
- private static final String KEYSPACE1 = "Keyspace1";
- private static final String KEYSPACE3 = "Keyspace3";
- private static final String KEYSPACE6 = "Keyspace6";
- private static final String EMPTYKEYSPACE = "DefsTestEmptyKeyspace";
- private static final String CF_STANDARD1 = "Standard1";
- private static final String CF_STANDARD2 = "Standard2";
- private static final String CF_INDEXED = "Indexed1";
-
- @BeforeClass
- public static void defineSchema() throws ConfigurationException
- {
- SchemaLoader.prepareServer();
- SchemaLoader.startGossiper();
- SchemaLoader.createKeyspace(KEYSPACE1,
- SimpleStrategy.class,
- KSMetaData.optsWithRF(1),
- SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
- SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
- SchemaLoader.createKeyspace(KEYSPACE3, true, false,
- SimpleStrategy.class,
- KSMetaData.optsWithRF(5),
- SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
- SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEXED, true));
- SchemaLoader.createKeyspace(KEYSPACE6,
- SimpleStrategy.class,
- KSMetaData.optsWithRF(1),
- SchemaLoader.indexCFMD(KEYSPACE6, CF_INDEXED, true));
- }
-
- @Test
- public void testCFMetaDataApply() throws ConfigurationException
- {
- CFMetaData cfm = new CFMetaData(KEYSPACE1,
- "TestApplyCFM_CF",
- ColumnFamilyType.Standard,
- new SimpleDenseCellNameType(BytesType.instance));
-
- for (int i = 0; i < 5; i++)
- {
- ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
- cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
- }
-
- cfm.comment("No comment")
- .readRepairChance(0.5)
- .gcGraceSeconds(100000)
- .minCompactionThreshold(500)
- .maxCompactionThreshold(500);
-
- // we'll be adding this one later. make sure it's not already there.
- Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
-
- CFMetaData cfNew = cfm.copy();
-
- // add one.
- ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
- .setIndex("5", IndexType.KEYS, null);
- cfNew.addColumnDefinition(addIndexDef);
-
- // remove one.
- ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
- .setIndex("0", IndexType.KEYS, null);
- Assert.assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
-
- cfm.apply(cfNew);
-
- for (int i = 1; i < cfm.allColumns().size(); i++)
- Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 1 })));
- Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 0 })));
- Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
- }
-
- @Test
- public void testInvalidNames()
- {
- String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
- for (String s : valid)
- Assert.assertTrue(CFMetaData.isNameValid(s));
-
- String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
- for (String s : invalid)
- Assert.assertFalse(CFMetaData.isNameValid(s));
- }
-
- @Ignore
- @Test
- public void saveAndRestore()
- {
- /*
- // verify dump and reload.
- UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
- DefsTables.dumpToStorage(first);
- List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
-
- Assert.assertTrue(defs.size() > 0);
- Assert.assertEquals(defs.size(), Schema.instance.getNonSystemKeyspaces().size());
- for (KSMetaData loaded : defs)
- {
- KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
- Assert.assertTrue(String.format("%s != %s", loaded, defined), defined.equals(loaded));
- }
- */
- }
-
- @Test
- public void addNewCfToBogusKeyspace()
- {
- CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
- try
- {
- MigrationManager.announceNewColumnFamily(newCf);
- throw new AssertionError("You shouldn't be able to do anything to a keyspace that doesn't exist.");
- }
- catch (ConfigurationException expected)
- {
- }
- }
-
- @Test
- public void addNewCfWithNullComment() throws ConfigurationException
- {
- final String ks = KEYSPACE1;
- final String cf = "BrandNewCfWithNull";
- KSMetaData original = Schema.instance.getKSMetaData(ks);
-
- CFMetaData newCf = addTestCF(original.name, cf, null);
-
- Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
- MigrationManager.announceNewColumnFamily(newCf);
-
- Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
- Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
- }
-
- @Test
- public void addNewCF() throws ConfigurationException
- {
- final String ks = KEYSPACE1;
- final String cf = "BrandNewCf";
- KSMetaData original = Schema.instance.getKSMetaData(ks);
-
- CFMetaData newCf = addTestCF(original.name, cf, "A New Table");
-
- Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
- MigrationManager.announceNewColumnFamily(newCf);
-
- Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
- Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
-
- // now read and write to it.
- CellName col0 = cellname("col0");
- DecoratedKey dk = Util.dk("key0");
- Mutation rm = new Mutation(ks, dk.getKey());
- rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
- rm.applyUnsafe();
- ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
- Assert.assertNotNull(store);
- store.forceBlockingFlush();
-
- ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
- Assert.assertNotNull(cfam.getColumn(col0));
- Cell col = cfam.getColumn(col0);
- Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
- }
-
- @Test
- public void dropCf() throws ConfigurationException
- {
- DecoratedKey dk = Util.dk("dropCf");
- // sanity
- final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
- Assert.assertNotNull(ks);
- final CFMetaData cfm = ks.cfMetaData().get("Standard1");
- Assert.assertNotNull(cfm);
-
- // write some data, force a flush, then verify that files exist on disk.
- Mutation rm = new Mutation(ks.name, dk.getKey());
- for (int i = 0; i < 100; i++)
- rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
- rm.applyUnsafe();
- ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
- Assert.assertNotNull(store);
- store.forceBlockingFlush();
- Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
-
- MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
-
- Assert.assertFalse(Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName));
-
- // any write should fail.
- rm = new Mutation(ks.name, dk.getKey());
- boolean success = true;
- try
- {
- rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
- rm.applyUnsafe();
- }
- catch (Throwable th)
- {
- success = false;
- }
- Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
-
- // verify that the files are gone.
- for (File file : store.directories.sstableLister().listFiles())
- {
- if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
- throw new AssertionError("undeleted file " + file);
- }
- }
-
- @Test
- public void addNewKS() throws ConfigurationException
- {
- DecoratedKey dk = Util.dk("key0");
- CFMetaData newCf = addTestCF("NewKeyspace1", "AddedStandard1", "A new cf for a new ks");
-
- KSMetaData newKs = KSMetaData.testMetadata(newCf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), newCf);
-
- MigrationManager.announceNewKeyspace(newKs);
-
- Assert.assertNotNull(Schema.instance.getKSMetaData(newCf.ksName));
- Assert.assertEquals(Schema.instance.getKSMetaData(newCf.ksName), newKs);
-
- // test reads and writes.
- CellName col0 = cellname("col0");
- Mutation rm = new Mutation(newCf.ksName, dk.getKey());
- rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
- rm.applyUnsafe();
- ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
- Assert.assertNotNull(store);
- store.forceBlockingFlush();
-
- ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
- Assert.assertNotNull(cfam.getColumn(col0));
- Cell col = cfam.getColumn(col0);
- Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
- }
-
- @Test
- public void dropKS() throws ConfigurationException
- {
- DecoratedKey dk = Util.dk("dropKs");
- // sanity
- final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
- Assert.assertNotNull(ks);
- final CFMetaData cfm = ks.cfMetaData().get("Standard2");
- Assert.assertNotNull(cfm);
-
- // write some data, force a flush, then verify that files exist on disk.
- Mutation rm = new Mutation(ks.name, dk.getKey());
- for (int i = 0; i < 100; i++)
- rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
- rm.applyUnsafe();
- ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
- Assert.assertNotNull(store);
- store.forceBlockingFlush();
- Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
-
- MigrationManager.announceKeyspaceDrop(ks.name);
-
- Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
-
- // write should fail.
- rm = new Mutation(ks.name, dk.getKey());
- boolean success = true;
- try
- {
- rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
- rm.applyUnsafe();
- }
- catch (Throwable th)
- {
- success = false;
- }
- Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
-
- // reads should fail too.
- boolean threw = false;
- try
- {
- Keyspace.open(ks.name);
- }
- catch (Throwable th)
- {
- threw = true;
- }
- Assert.assertTrue(threw);
- }
-
- @Test
- public void dropKSUnflushed() throws ConfigurationException
- {
- DecoratedKey dk = Util.dk("dropKs");
- // sanity
- final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE3);
- Assert.assertNotNull(ks);
- final CFMetaData cfm = ks.cfMetaData().get("Standard1");
- Assert.assertNotNull(cfm);
-
- // write some data
- Mutation rm = new Mutation(ks.name, dk.getKey());
- for (int i = 0; i < 100; i++)
- rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
- rm.applyUnsafe();
-
- MigrationManager.announceKeyspaceDrop(ks.name);
-
- Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
- }
-
- @Test
- public void createEmptyKsAddNewCf() throws ConfigurationException
- {
- Assert.assertNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
-
- KSMetaData newKs = KSMetaData.testMetadata(EMPTYKEYSPACE, SimpleStrategy.class, KSMetaData.optsWithRF(5));
-
- MigrationManager.announceNewKeyspace(newKs);
- Assert.assertNotNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
-
- CFMetaData newCf = addTestCF(EMPTYKEYSPACE, "AddedLater", "A new CF to add to an empty KS");
-
- //should not exist until apply
- Assert.assertFalse(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
-
- //add the new CF to the empty space
- MigrationManager.announceNewColumnFamily(newCf);
-
- Assert.assertTrue(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
- Assert.assertEquals(Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName), newCf);
-
- // now read and write to it.
- CellName col0 = cellname("col0");
- DecoratedKey dk = Util.dk("key0");
- Mutation rm = new Mutation(newKs.name, dk.getKey());
- rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
- rm.applyUnsafe();
- ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
- Assert.assertNotNull(store);
- store.forceBlockingFlush();
-
- ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
- Assert.assertNotNull(cfam.getColumn(col0));
- Cell col = cfam.getColumn(col0);
- Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
- }
-
- @Test
- public void testUpdateKeyspace() throws ConfigurationException
- {
- // create a keyspace to serve as existing.
- CFMetaData cf = addTestCF("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
- KSMetaData oldKs = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), cf);
-
- MigrationManager.announceNewKeyspace(oldKs);
-
- Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
- Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), oldKs);
-
- // names should match.
- KSMetaData newBadKs2 = KSMetaData.testMetadata(cf.ksName + "trash", SimpleStrategy.class, KSMetaData.optsWithRF(4));
- try
- {
- MigrationManager.announceKeyspaceUpdate(newBadKs2);
- throw new AssertionError("Should not have been able to update a KS with an invalid KS name.");
- }
- catch (ConfigurationException ex)
- {
- // expected.
- }
-
- KSMetaData newKs = KSMetaData.testMetadata(cf.ksName, OldNetworkTopologyStrategy.class, KSMetaData.optsWithRF(1));
- MigrationManager.announceKeyspaceUpdate(newKs);
-
- KSMetaData newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
- Assert.assertEquals(newFetchedKs.strategyClass, newKs.strategyClass);
- Assert.assertFalse(newFetchedKs.strategyClass.equals(oldKs.strategyClass));
- }
-
- @Test
- public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
- {
- // create a keyspace with a cf to update.
- CFMetaData cf = addTestCF("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
- KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
- MigrationManager.announceNewKeyspace(ksm);
-
- Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
- Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
- Assert.assertNotNull(Schema.instance.getCFMetaData(cf.ksName, cf.cfName));
-
- // updating certain fields should fail.
- CFMetaData newCfm = cf.copy();
- newCfm.defaultValidator(BytesType.instance);
- newCfm.minCompactionThreshold(5);
- newCfm.maxCompactionThreshold(31);
-
- // test valid operations.
- newCfm.comment("Modified comment");
- MigrationManager.announceColumnFamilyUpdate(newCfm, false); // doesn't get set back here.
-
- newCfm.readRepairChance(0.23);
- MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
- newCfm.gcGraceSeconds(12);
- MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
- newCfm.defaultValidator(UTF8Type.instance);
- MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
- newCfm.minCompactionThreshold(3);
- MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
- newCfm.maxCompactionThreshold(33);
- MigrationManager.announceColumnFamilyUpdate(newCfm, false);
-
- // can't test changing the reconciler because there is only one impl.
-
- // check the cumulative affect.
- Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
- Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
- Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
- Assert.assertEquals(UTF8Type.instance, Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getDefaultValidator());
-
- // Change cfId
- newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, cf.comparator);
- CFMetaData.copyOpts(newCfm, cf);
- try
- {
- cf.apply(newCfm);
- throw new AssertionError("Should have blown up when you used a different id.");
- }
- catch (ConfigurationException expected) {}
-
- // Change cfName
- newCfm = new CFMetaData(cf.ksName, cf.cfName + "_renamed", cf.cfType, cf.comparator);
- CFMetaData.copyOpts(newCfm, cf);
- try
- {
- cf.apply(newCfm);
- throw new AssertionError("Should have blown up when you used a different name.");
- }
- catch (ConfigurationException expected) {}
-
- // Change ksName
- newCfm = new CFMetaData(cf.ksName + "_renamed", cf.cfName, cf.cfType, cf.comparator);
- CFMetaData.copyOpts(newCfm, cf);
- try
- {
- cf.apply(newCfm);
- throw new AssertionError("Should have blown up when you used a different keyspace.");
- }
- catch (ConfigurationException expected) {}
-
- // Change cf type
- newCfm = new CFMetaData(cf.ksName, cf.cfName, ColumnFamilyType.Super, cf.comparator);
- CFMetaData.copyOpts(newCfm, cf);
- try
- {
- cf.apply(newCfm);
- throw new AssertionError("Should have blwon up when you used a different cf type.");
- }
- catch (ConfigurationException expected) {}
-
- // Change comparator
- newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
- CFMetaData.copyOpts(newCfm, cf);
- try
- {
- cf.apply(newCfm);
- throw new AssertionError("Should have blown up when you used a different comparator.");
- }
- catch (ConfigurationException expected) {}
- }
-
- @Test
- public void testDropIndex() throws ConfigurationException
- {
- // persist keyspace definition in the system keyspace
- Schema.instance.getKSMetaData(KEYSPACE6).toSchema(System.currentTimeMillis()).applyUnsafe();
- ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore("Indexed1");
-
- // insert some data. save the sstable descriptor so we can make sure it's marked for delete after the drop
- Mutation rm = new Mutation(KEYSPACE6, ByteBufferUtil.bytes("k1"));
- rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
- rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
- rm.applyUnsafe();
- cfs.forceBlockingFlush();
- ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
- Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
-
- // drop the index
- CFMetaData meta = cfs.metadata.copy();
- ColumnDefinition cdOld = meta.regularColumns().iterator().next();
- ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
- meta.addOrReplaceColumnDefinition(cdNew);
- MigrationManager.announceColumnFamilyUpdate(meta, false);
-
- // check
- Assert.assertTrue(cfs.indexManager.getIndexes().isEmpty());
- SSTableDeletingTask.waitForDeletions();
- Assert.assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
- }
-
- private CFMetaData addTestCF(String ks, String cf, String comment)
- {
- CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
- newCFMD.comment(comment)
- .readRepairChance(0.0);
-
- return newCFMD;
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/KSMetaDataTest.java b/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
index 9093086..3d90bef 100644
--- a/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/KSMetaDataTest.java
@@ -39,11 +39,7 @@ public class KSMetaDataTest
options.put("key2", "value2");
options.put("key3", "value3");
- KSMetaData ksMeta = new KSMetaData("test",
- SimpleStrategy.class,
- options,
- true,
- Collections.<CFMetaData>emptyList());
+ KSMetaData ksMeta = new KSMetaData("test", SimpleStrategy.class, options, true);
assertTrue(ksMeta.toString().contains(options.toString()));
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 883da3a..4e536fe 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -159,10 +159,10 @@ public abstract class CQLTester
schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tableToDrop));
for (String aggregateName : aggregatesToDrop)
- schemaChange(String.format("DROP AGGREGATE IF EXISTS %s", aggregateName));
+ schemaChange(String.format("DROP AGGREGATE IF EXISTS %s.%s", KEYSPACE, aggregateName));
for (String functionName : functionsToDrop)
- schemaChange(String.format("DROP FUNCTION IF EXISTS %s", functionName));
+ schemaChange(String.format("DROP FUNCTION IF EXISTS %s.%s", KEYSPACE, functionName));
for (String typeName : typesToDrop)
schemaChange(String.format("DROP TYPE IF EXISTS %s.%s", KEYSPACE, typeName));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
index 608aee8..70d1d0c 100644
--- a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
@@ -101,7 +101,7 @@ public class BatchlogManagerTest
}
// Flush the batchlog to disk (see CASSANDRA-6822).
- Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE).forceBlockingFlush();
+ Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG).forceBlockingFlush();
assertEquals(1000, BatchlogManager.instance.countAllBatches() - initialAllBatches);
assertEquals(0, BatchlogManager.instance.getTotalBatchesReplayed() - initialReplayedBatches);
@@ -171,7 +171,7 @@ public class BatchlogManagerTest
}
// Flush the batchlog to disk (see CASSANDRA-6822).
- Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE).forceFlush();
+ Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG).forceFlush();
// Force batchlog replay and wait for it to complete.
BatchlogManager.instance.startBatchlogReplay().get();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index 8b6834c..ce14c37 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -66,7 +66,7 @@ public class HintedHandOffTest
{
// prepare hints column family
Keyspace systemKeyspace = Keyspace.open("system");
- ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+ ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS);
hintStore.clearUnsafe();
hintStore.metadata.gcGraceSeconds(36000); // 10 hours
hintStore.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getCanonicalName());
@@ -101,7 +101,7 @@ public class HintedHandOffTest
HintedHandOffManager.instance.metrics.incrPastWindow(InetAddress.getLocalHost());
HintedHandOffManager.instance.metrics.log();
- UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS_TABLE);
+ UntypedResultSet rows = executeInternal("SELECT hints_dropped FROM system." + SystemKeyspace.PEER_EVENTS);
Map<UUID, Integer> returned = rows.one().getMap("hints_dropped", UUIDType.instance, Int32Type.instance);
assertEquals(Iterators.getLast(returned.values().iterator()).intValue(), 99);
}
@@ -110,7 +110,7 @@ public class HintedHandOffTest
public void testTruncateHints() throws Exception
{
Keyspace systemKeyspace = Keyspace.open("system");
- ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+ ColumnFamilyStore hintStore = systemKeyspace.getColumnFamilyStore(SystemKeyspace.HINTS);
hintStore.clearUnsafe();
// insert 1 hint
@@ -138,7 +138,7 @@ public class HintedHandOffTest
private int getNoOfHints()
{
String req = "SELECT * FROM system.%s";
- UntypedResultSet resultSet = executeInternal(String.format(req, SystemKeyspace.HINTS_TABLE));
+ UntypedResultSet resultSet = executeInternal(String.format(req, SystemKeyspace.HINTS));
return resultSet.size();
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java
new file mode 100644
index 0000000..8190617
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/LegacySchemaTablesTest.java
@@ -0,0 +1,568 @@
+/**
+ * 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.io.File;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.Util.cellname;
+
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+@RunWith(OrderedJUnit4ClassRunner.class)
+public class LegacySchemaTablesTest
+{
+ private static final String KEYSPACE1 = "Keyspace1";
+ private static final String KEYSPACE3 = "Keyspace3";
+ private static final String KEYSPACE6 = "Keyspace6";
+ private static final String EMPTYKEYSPACE = "DefsTestEmptyKeyspace";
+ private static final String CF_STANDARD1 = "Standard1";
+ private static final String CF_STANDARD2 = "Standard2";
+ private static final String CF_INDEXED = "Indexed1";
+
+ @BeforeClass
+ public static void defineSchema() throws ConfigurationException
+ {
+ SchemaLoader.prepareServer();
+ SchemaLoader.startGossiper();
+ SchemaLoader.createKeyspace(KEYSPACE1,
+ SimpleStrategy.class,
+ KSMetaData.optsWithRF(1),
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+ SchemaLoader.createKeyspace(KEYSPACE3, true, false,
+ SimpleStrategy.class,
+ KSMetaData.optsWithRF(5),
+ SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+ SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEXED, true));
+ SchemaLoader.createKeyspace(KEYSPACE6,
+ SimpleStrategy.class,
+ KSMetaData.optsWithRF(1),
+ SchemaLoader.indexCFMD(KEYSPACE6, CF_INDEXED, true));
+ }
+
+ @Test
+ public void testCFMetaDataApply() throws ConfigurationException
+ {
+ CFMetaData cfm = new CFMetaData(KEYSPACE1,
+ "TestApplyCFM_CF",
+ ColumnFamilyType.Standard,
+ new SimpleDenseCellNameType(BytesType.instance));
+
+ for (int i = 0; i < 5; i++)
+ {
+ ByteBuffer name = ByteBuffer.wrap(new byte[] { (byte)i });
+ cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, name, BytesType.instance, null).setIndex(Integer.toString(i), IndexType.KEYS, null));
+ }
+
+ cfm.comment("No comment")
+ .readRepairChance(0.5)
+ .gcGraceSeconds(100000)
+ .minCompactionThreshold(500)
+ .maxCompactionThreshold(500);
+
+ // we'll be adding this one later. make sure it's not already there.
+ Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
+
+ CFMetaData cfNew = cfm.copy();
+
+ // add one.
+ ColumnDefinition addIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 5 }), BytesType.instance, null)
+ .setIndex("5", IndexType.KEYS, null);
+ cfNew.addColumnDefinition(addIndexDef);
+
+ // remove one.
+ ColumnDefinition removeIndexDef = ColumnDefinition.regularDef(cfm, ByteBuffer.wrap(new byte[] { 0 }), BytesType.instance, null)
+ .setIndex("0", IndexType.KEYS, null);
+ Assert.assertTrue(cfNew.removeColumnDefinition(removeIndexDef));
+
+ cfm.apply(cfNew);
+
+ for (int i = 1; i < cfm.allColumns().size(); i++)
+ Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 1 })));
+ Assert.assertNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 0 })));
+ Assert.assertNotNull(cfm.getColumnDefinition(ByteBuffer.wrap(new byte[] { 5 })));
+ }
+
+ @Test
+ public void testInvalidNames()
+ {
+ String[] valid = {"1", "a", "_1", "b_", "__", "1_a"};
+ for (String s : valid)
+ Assert.assertTrue(CFMetaData.isNameValid(s));
+
+ String[] invalid = {"b@t", "dash-y", "", " ", "dot.s", ".hidden"};
+ for (String s : invalid)
+ Assert.assertFalse(CFMetaData.isNameValid(s));
+ }
+
+ @Ignore
+ @Test
+ public void saveAndRestore()
+ {
+ /*
+ // verify dump and reload.
+ UUID first = UUIDGen.makeType1UUIDFromHost(FBUtilities.getBroadcastAddress());
+ DefsTables.dumpToStorage(first);
+ List<KSMetaData> defs = new ArrayList<KSMetaData>(DefsTables.loadFromStorage(first));
+
+ Assert.assertTrue(defs.size() > 0);
+ Assert.assertEquals(defs.size(), Schema.instance.getNonSystemKeyspaces().size());
+ for (KSMetaData loaded : defs)
+ {
+ KSMetaData defined = Schema.instance.getKeyspaceDefinition(loaded.name);
+ Assert.assertTrue(String.format("%s != %s", loaded, defined), defined.equals(loaded));
+ }
+ */
+ }
+
+ @Test
+ public void addNewCfToBogusKeyspace()
+ {
+ CFMetaData newCf = addTestCF("MadeUpKeyspace", "NewCF", "new cf");
+ try
+ {
+ MigrationManager.announceNewColumnFamily(newCf);
+ throw new AssertionError("You shouldn't be able to do anything to a keyspace that doesn't exist.");
+ }
+ catch (ConfigurationException expected)
+ {
+ }
+ }
+
+ @Test
+ public void addNewCfWithNullComment() throws ConfigurationException
+ {
+ final String ks = KEYSPACE1;
+ final String cf = "BrandNewCfWithNull";
+ KSMetaData original = Schema.instance.getKSMetaData(ks);
+
+ CFMetaData newCf = addTestCF(original.name, cf, null);
+
+ Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+ MigrationManager.announceNewColumnFamily(newCf);
+
+ Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+ Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
+ }
+
+ @Test
+ public void addNewCF() throws ConfigurationException
+ {
+ final String ks = KEYSPACE1;
+ final String cf = "BrandNewCf";
+ KSMetaData original = Schema.instance.getKSMetaData(ks);
+
+ CFMetaData newCf = addTestCF(original.name, cf, "A New Table");
+
+ Assert.assertFalse(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+ MigrationManager.announceNewColumnFamily(newCf);
+
+ Assert.assertTrue(Schema.instance.getKSMetaData(ks).cfMetaData().containsKey(newCf.cfName));
+ Assert.assertEquals(newCf, Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName));
+
+ // now read and write to it.
+ CellName col0 = cellname("col0");
+ DecoratedKey dk = Util.dk("key0");
+ Mutation rm = new Mutation(ks, dk.getKey());
+ rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
+ rm.applyUnsafe();
+ ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
+ Assert.assertNotNull(store);
+ store.forceBlockingFlush();
+
+ ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+ Assert.assertNotNull(cfam.getColumn(col0));
+ Cell col = cfam.getColumn(col0);
+ Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+ }
+
+ @Test
+ public void dropCf() throws ConfigurationException
+ {
+ DecoratedKey dk = Util.dk("dropCf");
+ // sanity
+ final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
+ Assert.assertNotNull(ks);
+ final CFMetaData cfm = ks.cfMetaData().get("Standard1");
+ Assert.assertNotNull(cfm);
+
+ // write some data, force a flush, then verify that files exist on disk.
+ Mutation rm = new Mutation(ks.name, dk.getKey());
+ for (int i = 0; i < 100; i++)
+ rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+ rm.applyUnsafe();
+ ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+ Assert.assertNotNull(store);
+ store.forceBlockingFlush();
+ Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
+
+ MigrationManager.announceColumnFamilyDrop(ks.name, cfm.cfName);
+
+ Assert.assertFalse(Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName));
+
+ // any write should fail.
+ rm = new Mutation(ks.name, dk.getKey());
+ boolean success = true;
+ try
+ {
+ rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
+ rm.applyUnsafe();
+ }
+ catch (Throwable th)
+ {
+ success = false;
+ }
+ Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+ // verify that the files are gone.
+ for (File file : store.directories.sstableLister().listFiles())
+ {
+ if (file.getPath().endsWith("Data.db") && !new File(file.getPath().replace("Data.db", "Compacted")).exists())
+ throw new AssertionError("undeleted file " + file);
+ }
+ }
+
+ @Test
+ public void addNewKS() throws ConfigurationException
+ {
+ DecoratedKey dk = Util.dk("key0");
+ CFMetaData newCf = addTestCF("NewKeyspace1", "AddedStandard1", "A new cf for a new ks");
+
+ KSMetaData newKs = KSMetaData.testMetadata(newCf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), newCf);
+
+ MigrationManager.announceNewKeyspace(newKs);
+
+ Assert.assertNotNull(Schema.instance.getKSMetaData(newCf.ksName));
+ Assert.assertEquals(Schema.instance.getKSMetaData(newCf.ksName), newKs);
+
+ // test reads and writes.
+ CellName col0 = cellname("col0");
+ Mutation rm = new Mutation(newCf.ksName, dk.getKey());
+ rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
+ rm.applyUnsafe();
+ ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
+ Assert.assertNotNull(store);
+ store.forceBlockingFlush();
+
+ ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+ Assert.assertNotNull(cfam.getColumn(col0));
+ Cell col = cfam.getColumn(col0);
+ Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+ }
+
+ @Test
+ public void dropKS() throws ConfigurationException
+ {
+ DecoratedKey dk = Util.dk("dropKs");
+ // sanity
+ final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
+ Assert.assertNotNull(ks);
+ final CFMetaData cfm = ks.cfMetaData().get("Standard2");
+ Assert.assertNotNull(cfm);
+
+ // write some data, force a flush, then verify that files exist on disk.
+ Mutation rm = new Mutation(ks.name, dk.getKey());
+ for (int i = 0; i < 100; i++)
+ rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+ rm.applyUnsafe();
+ ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
+ Assert.assertNotNull(store);
+ store.forceBlockingFlush();
+ Assert.assertTrue(store.directories.sstableLister().list().size() > 0);
+
+ MigrationManager.announceKeyspaceDrop(ks.name);
+
+ Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
+
+ // write should fail.
+ rm = new Mutation(ks.name, dk.getKey());
+ boolean success = true;
+ try
+ {
+ rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
+ rm.applyUnsafe();
+ }
+ catch (Throwable th)
+ {
+ success = false;
+ }
+ Assert.assertFalse("This mutation should have failed since the CF no longer exists.", success);
+
+ // reads should fail too.
+ boolean threw = false;
+ try
+ {
+ Keyspace.open(ks.name);
+ }
+ catch (Throwable th)
+ {
+ threw = true;
+ }
+ Assert.assertTrue(threw);
+ }
+
+ @Test
+ public void dropKSUnflushed() throws ConfigurationException
+ {
+ DecoratedKey dk = Util.dk("dropKs");
+ // sanity
+ final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE3);
+ Assert.assertNotNull(ks);
+ final CFMetaData cfm = ks.cfMetaData().get("Standard1");
+ Assert.assertNotNull(cfm);
+
+ // write some data
+ Mutation rm = new Mutation(ks.name, dk.getKey());
+ for (int i = 0; i < 100; i++)
+ rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
+ rm.applyUnsafe();
+
+ MigrationManager.announceKeyspaceDrop(ks.name);
+
+ Assert.assertNull(Schema.instance.getKSMetaData(ks.name));
+ }
+
+ @Test
+ public void createEmptyKsAddNewCf() throws ConfigurationException
+ {
+ Assert.assertNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
+
+ KSMetaData newKs = KSMetaData.testMetadata(EMPTYKEYSPACE, SimpleStrategy.class, KSMetaData.optsWithRF(5));
+
+ MigrationManager.announceNewKeyspace(newKs);
+ Assert.assertNotNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
+
+ CFMetaData newCf = addTestCF(EMPTYKEYSPACE, "AddedLater", "A new CF to add to an empty KS");
+
+ //should not exist until apply
+ Assert.assertFalse(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
+
+ //add the new CF to the empty space
+ MigrationManager.announceNewColumnFamily(newCf);
+
+ Assert.assertTrue(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
+ Assert.assertEquals(Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName), newCf);
+
+ // now read and write to it.
+ CellName col0 = cellname("col0");
+ DecoratedKey dk = Util.dk("key0");
+ Mutation rm = new Mutation(newKs.name, dk.getKey());
+ rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
+ rm.applyUnsafe();
+ ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
+ Assert.assertNotNull(store);
+ store.forceBlockingFlush();
+
+ ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+ Assert.assertNotNull(cfam.getColumn(col0));
+ Cell col = cfam.getColumn(col0);
+ Assert.assertEquals(ByteBufferUtil.bytes("value0"), col.value());
+ }
+
+ @Test
+ public void testUpdateKeyspace() throws ConfigurationException
+ {
+ // create a keyspace to serve as existing.
+ CFMetaData cf = addTestCF("UpdatedKeyspace", "AddedStandard1", "A new cf for a new ks");
+ KSMetaData oldKs = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(5), cf);
+
+ MigrationManager.announceNewKeyspace(oldKs);
+
+ Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+ Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), oldKs);
+
+ // names should match.
+ KSMetaData newBadKs2 = KSMetaData.testMetadata(cf.ksName + "trash", SimpleStrategy.class, KSMetaData.optsWithRF(4));
+ try
+ {
+ MigrationManager.announceKeyspaceUpdate(newBadKs2);
+ throw new AssertionError("Should not have been able to update a KS with an invalid KS name.");
+ }
+ catch (ConfigurationException ex)
+ {
+ // expected.
+ }
+
+ KSMetaData newKs = KSMetaData.testMetadata(cf.ksName, OldNetworkTopologyStrategy.class, KSMetaData.optsWithRF(1));
+ MigrationManager.announceKeyspaceUpdate(newKs);
+
+ KSMetaData newFetchedKs = Schema.instance.getKSMetaData(newKs.name);
+ Assert.assertEquals(newFetchedKs.strategyClass, newKs.strategyClass);
+ Assert.assertFalse(newFetchedKs.strategyClass.equals(oldKs.strategyClass));
+ }
+
+ @Test
+ public void testUpdateColumnFamilyNoIndexes() throws ConfigurationException
+ {
+ // create a keyspace with a cf to update.
+ CFMetaData cf = addTestCF("UpdatedCfKs", "Standard1added", "A new cf that will be updated");
+ KSMetaData ksm = KSMetaData.testMetadata(cf.ksName, SimpleStrategy.class, KSMetaData.optsWithRF(1), cf);
+ MigrationManager.announceNewKeyspace(ksm);
+
+ Assert.assertNotNull(Schema.instance.getKSMetaData(cf.ksName));
+ Assert.assertEquals(Schema.instance.getKSMetaData(cf.ksName), ksm);
+ Assert.assertNotNull(Schema.instance.getCFMetaData(cf.ksName, cf.cfName));
+
+ // updating certain fields should fail.
+ CFMetaData newCfm = cf.copy();
+ newCfm.defaultValidator(BytesType.instance);
+ newCfm.minCompactionThreshold(5);
+ newCfm.maxCompactionThreshold(31);
+
+ // test valid operations.
+ newCfm.comment("Modified comment");
+ MigrationManager.announceColumnFamilyUpdate(newCfm, false); // doesn't get set back here.
+
+ newCfm.readRepairChance(0.23);
+ MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+ newCfm.gcGraceSeconds(12);
+ MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+ newCfm.defaultValidator(UTF8Type.instance);
+ MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+ newCfm.minCompactionThreshold(3);
+ MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+ newCfm.maxCompactionThreshold(33);
+ MigrationManager.announceColumnFamilyUpdate(newCfm, false);
+
+ // can't test changing the reconciler because there is only one impl.
+
+ // check the cumulative affect.
+ Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getComment(), newCfm.getComment());
+ Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getReadRepairChance(), newCfm.getReadRepairChance(), 0.0001);
+ Assert.assertEquals(Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getGcGraceSeconds(), newCfm.getGcGraceSeconds());
+ Assert.assertEquals(UTF8Type.instance, Schema.instance.getCFMetaData(cf.ksName, cf.cfName).getDefaultValidator());
+
+ // Change cfId
+ newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, cf.comparator);
+ CFMetaData.copyOpts(newCfm, cf);
+ try
+ {
+ cf.apply(newCfm);
+ throw new AssertionError("Should have blown up when you used a different id.");
+ }
+ catch (ConfigurationException expected) {}
+
+ // Change cfName
+ newCfm = new CFMetaData(cf.ksName, cf.cfName + "_renamed", cf.cfType, cf.comparator);
+ CFMetaData.copyOpts(newCfm, cf);
+ try
+ {
+ cf.apply(newCfm);
+ throw new AssertionError("Should have blown up when you used a different name.");
+ }
+ catch (ConfigurationException expected) {}
+
+ // Change ksName
+ newCfm = new CFMetaData(cf.ksName + "_renamed", cf.cfName, cf.cfType, cf.comparator);
+ CFMetaData.copyOpts(newCfm, cf);
+ try
+ {
+ cf.apply(newCfm);
+ throw new AssertionError("Should have blown up when you used a different keyspace.");
+ }
+ catch (ConfigurationException expected) {}
+
+ // Change cf type
+ newCfm = new CFMetaData(cf.ksName, cf.cfName, ColumnFamilyType.Super, cf.comparator);
+ CFMetaData.copyOpts(newCfm, cf);
+ try
+ {
+ cf.apply(newCfm);
+ throw new AssertionError("Should have blwon up when you used a different cf type.");
+ }
+ catch (ConfigurationException expected) {}
+
+ // Change comparator
+ newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
+ CFMetaData.copyOpts(newCfm, cf);
+ try
+ {
+ cf.apply(newCfm);
+ throw new AssertionError("Should have blown up when you used a different comparator.");
+ }
+ catch (ConfigurationException expected) {}
+ }
+
+ @Test
+ public void testDropIndex() throws ConfigurationException
+ {
+ // persist keyspace definition in the system keyspace
+ LegacySchemaTables.makeCreateKeyspaceMutation(Schema.instance.getKSMetaData(KEYSPACE6), FBUtilities.timestampMicros()).applyUnsafe();
+ ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore("Indexed1");
+
+ // insert some data. save the sstable descriptor so we can make sure it's marked for delete after the drop
+ Mutation rm = new Mutation(KEYSPACE6, ByteBufferUtil.bytes("k1"));
+ rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
+ rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
+ rm.applyUnsafe();
+ cfs.forceBlockingFlush();
+ ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
+ Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
+
+ // drop the index
+ CFMetaData meta = cfs.metadata.copy();
+ ColumnDefinition cdOld = meta.regularColumns().iterator().next();
+ ColumnDefinition cdNew = ColumnDefinition.regularDef(meta, cdOld.name.bytes, cdOld.type, null);
+ meta.addOrReplaceColumnDefinition(cdNew);
+ MigrationManager.announceColumnFamilyUpdate(meta, false);
+
+ // check
+ Assert.assertTrue(cfs.indexManager.getIndexes().isEmpty());
+ SSTableDeletingTask.waitForDeletions();
+ Assert.assertFalse(new File(desc.filenameFor(Component.DATA)).exists());
+ }
+
+ private CFMetaData addTestCF(String ks, String cf, String comment)
+ {
+ CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
+ newCFMD.comment(comment)
+ .readRepairChance(0.0);
+
+ return newCFMD;
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
index 1437886..ed0efee 100644
--- a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
@@ -59,11 +59,11 @@ public class EmbeddedCassandraServiceTest
public static void defineSchema() throws Exception
{
SchemaLoader.prepareServer();
+ setup();
SchemaLoader.createKeyspace(KEYSPACE1,
SimpleStrategy.class,
KSMetaData.optsWithRF(1),
SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
- setup();
}
/**
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index f98751a..d1a4d85 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -37,6 +37,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
@@ -98,7 +99,7 @@ public class StorageServiceServerTest
public void testColumnFamilySnapshot() throws IOException
{
// no need to insert extra data, even an "empty" database will have a little information in the system keyspace
- StorageService.instance.takeColumnFamilySnapshot(SystemKeyspace.NAME, SystemKeyspace.SCHEMA_KEYSPACES_TABLE, "cf_snapshot");
+ StorageService.instance.takeColumnFamilySnapshot(SystemKeyspace.NAME, LegacySchemaTables.KEYSPACES, "cf_snapshot");
}
@Test
[4/5] cassandra git commit: Isolate schema serializaton code
Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
index 8b42e51..4672451 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDFunction.java
@@ -30,11 +30,7 @@ import com.datastax.driver.core.UserType;
import org.apache.cassandra.config.KSMetaData;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.db.marshal.TypeParser;
import org.apache.cassandra.exceptions.*;
import org.apache.cassandra.service.MigrationManager;
import org.apache.cassandra.utils.ByteBufferUtil;
@@ -47,9 +43,10 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
protected static final Logger logger = LoggerFactory.getLogger(UDFunction.class);
protected final List<ColumnIdentifier> argNames;
+
protected final String language;
protected final String body;
- private final boolean deterministic;
+ protected final boolean isDeterministic;
protected final DataType[] argDataTypes;
protected final DataType returnDataType;
@@ -60,10 +57,10 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
AbstractType<?> returnType,
String language,
String body,
- boolean deterministic)
+ boolean isDeterministic)
{
this(name, argNames, argTypes, UDHelper.driverTypes(argTypes), returnType,
- UDHelper.driverType(returnType), language, body, deterministic);
+ UDHelper.driverType(returnType), language, body, isDeterministic);
}
protected UDFunction(FunctionName name,
@@ -74,14 +71,14 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
DataType returnDataType,
String language,
String body,
- boolean deterministic)
+ boolean isDeterministic)
{
super(name, argTypes, returnType);
assert new HashSet<>(argNames).size() == argNames.size() : "duplicate argument names";
this.argNames = argNames;
this.language = language;
this.body = body;
- this.deterministic = deterministic;
+ this.isDeterministic = isDeterministic;
this.argDataTypes = argDataTypes;
this.returnDataType = returnDataType;
}
@@ -92,13 +89,13 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
AbstractType<?> returnType,
String language,
String body,
- boolean deterministic)
+ boolean isDeterministic)
throws InvalidRequestException
{
switch (language)
{
- case "java": return JavaSourceUDFFactory.buildUDF(name, argNames, argTypes, returnType, body, deterministic);
- default: return new ScriptBasedUDF(name, argNames, argTypes, returnType, language, body, deterministic);
+ case "java": return JavaSourceUDFFactory.buildUDF(name, argNames, argTypes, returnType, body, isDeterministic);
+ default: return new ScriptBasedUDF(name, argNames, argTypes, returnType, language, body, isDeterministic);
}
}
@@ -111,24 +108,27 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
* 2) we return a meaningful error message if the function is executed (something more precise
* than saying that the function doesn't exist)
*/
- private static UDFunction createBrokenFunction(FunctionName name,
- List<ColumnIdentifier> argNames,
- List<AbstractType<?>> argTypes,
- AbstractType<?> returnType,
- String language,
- String body,
- final InvalidRequestException reason)
+ public static UDFunction createBrokenFunction(FunctionName name,
+ List<ColumnIdentifier> argNames,
+ List<AbstractType<?>> argTypes,
+ AbstractType<?> returnType,
+ String language,
+ String body,
+ final InvalidRequestException reason)
{
return new UDFunction(name, argNames, argTypes, returnType, language, body, true)
{
public ByteBuffer execute(int protocolVersion, List<ByteBuffer> parameters) throws InvalidRequestException
{
- throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully for the following reason: %s. "
- + "Please see the server log for more details", this, reason.getMessage()));
+ throw new InvalidRequestException(String.format("Function '%s' exists but hasn't been loaded successfully "
+ + "for the following reason: %s. Please see the server log for details",
+ this,
+ reason.getMessage()));
}
};
}
+
public boolean isAggregate()
{
return false;
@@ -136,7 +136,7 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
public boolean isPure()
{
- return deterministic;
+ return isDeterministic;
}
public boolean isNative()
@@ -144,13 +144,33 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
return false;
}
+ public List<ColumnIdentifier> argNames()
+ {
+ return argNames;
+ }
+
+ public boolean isDeterministic()
+ {
+ return isDeterministic;
+ }
+
+ public String body()
+ {
+ return body;
+ }
+
+ public String language()
+ {
+ return language;
+ }
+
/**
* Used by UDF implementations (both Java code generated by {@link org.apache.cassandra.cql3.functions.JavaSourceUDFFactory}
* and script executor {@link org.apache.cassandra.cql3.functions.ScriptBasedUDF}) to convert the C*
* serialized representation to the Java object representation.
*
* @param protocolVersion the native protocol version used for serialization
- * @param argIndex index of the UDF input argument
+ * @param argIndex index of the UDF input argument
*/
protected Object compose(int protocolVersion, int argIndex, ByteBuffer value)
{
@@ -169,117 +189,6 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
return value == null ? null : returnDataType.serialize(value, ProtocolVersion.fromInt(protocolVersion));
}
- private static Mutation makeSchemaMutation(FunctionName name)
- {
- UTF8Type kv = (UTF8Type)SystemKeyspace.SchemaFunctionsTable.getKeyValidator();
- return new Mutation(SystemKeyspace.NAME, kv.decompose(name.keyspace));
- }
-
- public Mutation toSchemaDrop(long timestamp)
- {
- Mutation mutation = makeSchemaMutation(name);
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
-
- Composite prefix = SystemKeyspace.SchemaFunctionsTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
- int ldt = (int) (System.currentTimeMillis() / 1000);
- cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
- return mutation;
- }
-
- public static Map<Composite, UDFunction> fromSchema(Row row)
- {
- UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, row);
- Map<Composite, UDFunction> udfs = new HashMap<>(results.size());
- for (UntypedResultSet.Row result : results)
- udfs.put(SystemKeyspace.SchemaFunctionsTable.comparator.make(result.getString("function_name"), result.getBlob("signature")),
- fromSchema(result));
- return udfs;
- }
-
- public Mutation toSchemaUpdate(long timestamp)
- {
- Mutation mutation = makeSchemaMutation(name);
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE);
-
- Composite prefix = SystemKeyspace.SchemaFunctionsTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
- CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
- adder.resetCollection("argument_names");
- adder.resetCollection("argument_types");
- adder.add("return_type", returnType.toString());
- adder.add("language", language);
- adder.add("body", body);
- adder.add("deterministic", deterministic);
-
- for (int i = 0; i < argNames.size(); i++)
- {
- adder.addListEntry("argument_names", argNames.get(i).bytes);
- adder.addListEntry("argument_types", argTypes.get(i).toString());
- }
-
- return mutation;
- }
-
- public static UDFunction fromSchema(UntypedResultSet.Row row)
- {
- String ksName = row.getString("keyspace_name");
- String functionName = row.getString("function_name");
- FunctionName name = new FunctionName(ksName, functionName);
-
- List<String> names = row.getList("argument_names", UTF8Type.instance);
- List<String> types = row.getList("argument_types", UTF8Type.instance);
-
- List<ColumnIdentifier> argNames;
- if (names == null)
- argNames = Collections.emptyList();
- else
- {
- argNames = new ArrayList<>(names.size());
- for (String arg : names)
- argNames.add(new ColumnIdentifier(arg, true));
- }
-
- List<AbstractType<?>> argTypes;
- if (types == null)
- argTypes = Collections.emptyList();
- else
- {
- argTypes = new ArrayList<>(types.size());
- for (String type : types)
- argTypes.add(parseType(type));
- }
-
- AbstractType<?> returnType = parseType(row.getString("return_type"));
-
- boolean deterministic = row.getBoolean("deterministic");
- String language = row.getString("language");
- String body = row.getString("body");
-
- try
- {
- return create(name, argNames, argTypes, returnType, language, body, deterministic);
- }
- catch (InvalidRequestException e)
- {
- logger.error(String.format("Cannot load function '%s' from schema: this function won't be available (on this node)", name), e);
- return createBrokenFunction(name, argNames, argTypes, returnType, language, body, e);
- }
- }
-
- private static AbstractType<?> parseType(String str)
- {
- // We only use this when reading the schema where we shouldn't get an error
- try
- {
- return TypeParser.parse(str);
- }
- catch (SyntaxException | ConfigurationException e)
- {
- throw new RuntimeException(e);
- }
- }
-
@Override
public boolean equals(Object o)
{
@@ -287,19 +196,19 @@ public abstract class UDFunction extends AbstractFunction implements ScalarFunct
return false;
UDFunction that = (UDFunction)o;
- return Objects.equal(this.name, that.name)
- && Functions.typeEquals(this.argTypes, that.argTypes)
- && Functions.typeEquals(this.returnType, that.returnType)
- && Objects.equal(this.argNames, that.argNames)
- && Objects.equal(this.language, that.language)
- && Objects.equal(this.body, that.body)
- && Objects.equal(this.deterministic, that.deterministic);
+ return Objects.equal(name, that.name)
+ && Objects.equal(argNames, that.argNames)
+ && Functions.typeEquals(argTypes, that.argTypes)
+ && Functions.typeEquals(returnType, that.returnType)
+ && Objects.equal(language, that.language)
+ && Objects.equal(body, that.body)
+ && Objects.equal(isDeterministic, that.isDeterministic);
}
@Override
public int hashCode()
{
- return Objects.hashCode(name, argTypes, returnType, argNames, language, body, deterministic);
+ return Objects.hashCode(name, argNames, argTypes, returnType, language, body, isDeterministic);
}
public void userTypeUpdated(String ksName, String typeName)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
index 2a17c75..0738cbe 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDHelper.java
@@ -31,12 +31,13 @@ import org.slf4j.LoggerFactory;
import com.datastax.driver.core.DataType;
import org.apache.cassandra.cql3.*;
import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.utils.FBUtilities;
/**
* Helper class for User Defined Functions + Aggregates.
*/
-final class UDHelper
+public final class UDHelper
{
protected static final Logger logger = LoggerFactory.getLogger(UDHelper.class);
@@ -112,12 +113,13 @@ final class UDHelper
// we use a "signature" which is just a SHA-1 of it's argument types (we could replace that by
// using a "signature" UDT that would be comprised of the function name and argument types,
// which we could then use as clustering column. But as we haven't yet used UDT in system tables,
- // We'll left that decision to #6717).
- protected static ByteBuffer computeSignature(List<AbstractType<?>> argTypes)
+ // We'll leave that decision to #6717).
+ public static ByteBuffer calculateSignature(AbstractFunction fun)
{
MessageDigest digest = FBUtilities.newMessageDigest("SHA-1");
- for (AbstractType<?> type : argTypes)
- digest.update(type.asCQL3Type().toString().getBytes(StandardCharsets.UTF_8));
+ digest.update(UTF8Type.instance.decompose(fun.name().name));
+ for (AbstractType<?> type : fun.argTypes())
+ digest.update(UTF8Type.instance.decompose(type.asCQL3Type().toString()));
return ByteBuffer.wrap(digest.digest());
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
index 6aea3b1..c8c2474 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTableStatement.java
@@ -151,14 +151,32 @@ public class CreateTableStatement extends SchemaAlteringStatement
.addAllColumnDefinitions(getColumns(cfmd))
.isDense(isDense);
- cfmd.addColumnMetadataFromAliases(keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
- cfmd.addColumnMetadataFromAliases(columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
+ addColumnMetadataFromAliases(cfmd, keyAliases, keyValidator, ColumnDefinition.Kind.PARTITION_KEY);
+ addColumnMetadataFromAliases(cfmd, columnAliases, comparator.asAbstractType(), ColumnDefinition.Kind.CLUSTERING_COLUMN);
if (valueAlias != null)
- cfmd.addColumnMetadataFromAliases(Collections.<ByteBuffer>singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
+ addColumnMetadataFromAliases(cfmd, Collections.singletonList(valueAlias), defaultValidator, ColumnDefinition.Kind.COMPACT_VALUE);
properties.applyToCFMetadata(cfmd);
}
+ private void addColumnMetadataFromAliases(CFMetaData cfm, List<ByteBuffer> aliases, AbstractType<?> comparator, ColumnDefinition.Kind kind)
+ {
+ if (comparator instanceof CompositeType)
+ {
+ CompositeType ct = (CompositeType)comparator;
+ for (int i = 0; i < aliases.size(); ++i)
+ if (aliases.get(i) != null)
+ cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(i), ct.types.get(i), i, kind));
+ }
+ else
+ {
+ assert aliases.size() <= 1;
+ if (!aliases.isEmpty() && aliases.get(0) != null)
+ cfm.addOrReplaceColumnDefinition(new ColumnDefinition(cfm, aliases.get(0), comparator, null, kind));
+ }
+ }
+
+
public static class RawStatement extends CFStatement
{
private final Map<ColumnIdentifier, CQL3Type.Raw> definitions = new HashMap<>();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
index 1fcd63c..e766f65 100644
--- a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.db.composites.CellName;
import org.apache.cassandra.db.composites.Composite;
import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.marshal.BytesType;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.SearchIterator;
import org.apache.cassandra.utils.btree.BTree;
@@ -59,7 +60,7 @@ import static org.apache.cassandra.db.index.SecondaryIndexManager.Updater;
*/
public class AtomicBTreeColumns extends ColumnFamily
{
- static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(SystemKeyspace.BuiltIndexesTable, null))
+ static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.denseCFMetaData("keyspace", "table", BytesType.instance), null))
+ ObjectSizes.measure(new Holder(null, null));
// Reserved values for wasteTracker field. These values must not be consecutive (see avoidReservedValues)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index b33e457..e71a62c 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -102,7 +102,7 @@ public class BatchlogManager implements BatchlogManagerMBean
public int countAllBatches()
{
- String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHLOG_TABLE);
+ String query = String.format("SELECT count(*) FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.BATCHLOG);
return (int) executeInternal(query).one().getLong("count");
}
@@ -137,8 +137,8 @@ public class BatchlogManager implements BatchlogManagerMBean
@VisibleForTesting
static Mutation getBatchlogMutationFor(Collection<Mutation> mutations, UUID uuid, int version, long now)
{
- ColumnFamily cf = ArrayBackedSortedColumns.factory.create(SystemKeyspace.BatchlogTable);
- CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.BatchlogTable.comparator.builder().build(), now);
+ ColumnFamily cf = ArrayBackedSortedColumns.factory.create(SystemKeyspace.Batchlog);
+ CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.Batchlog.comparator.builder().build(), now);
adder.add("data", serializeMutations(mutations, version))
.add("written_at", new Date(now / 1000))
.add("version", version);
@@ -174,7 +174,7 @@ public class BatchlogManager implements BatchlogManagerMBean
UntypedResultSet page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s LIMIT %d",
SystemKeyspace.NAME,
- SystemKeyspace.BATCHLOG_TABLE,
+ SystemKeyspace.BATCHLOG,
PAGE_SIZE));
while (!page.isEmpty())
@@ -186,7 +186,7 @@ public class BatchlogManager implements BatchlogManagerMBean
page = executeInternal(String.format("SELECT id, data, written_at, version FROM %s.%s WHERE token(id) > token(?) LIMIT %d",
SystemKeyspace.NAME,
- SystemKeyspace.BATCHLOG_TABLE,
+ SystemKeyspace.BATCHLOG,
PAGE_SIZE),
id);
}
@@ -199,7 +199,7 @@ public class BatchlogManager implements BatchlogManagerMBean
private void deleteBatch(UUID id)
{
Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(id));
- mutation.delete(SystemKeyspace.BATCHLOG_TABLE, FBUtilities.timestampMicros());
+ mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
mutation.apply();
}
@@ -447,7 +447,7 @@ public class BatchlogManager implements BatchlogManagerMBean
// force flush + compaction to reclaim space from the replayed batches
private void cleanup() throws ExecutionException, InterruptedException
{
- ColumnFamilyStore cfs = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG_TABLE);
+ ColumnFamilyStore cfs = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHLOG);
cfs.forceBlockingFlush();
Collection<Descriptor> descriptors = new ArrayList<>();
for (SSTableReader sstr : cfs.getSSTables())
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
index 5cb62ed..d5ede03 100644
--- a/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/DefinitionsUpdateVerbHandler.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.net.IVerbHandler;
import org.apache.cassandra.net.MessageIn;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.utils.WrappedRunnable;
/**
@@ -46,7 +47,7 @@ public class DefinitionsUpdateVerbHandler implements IVerbHandler<Collection<Mut
{
public void runMayThrow() throws Exception
{
- DefsTables.mergeSchema(message.payload);
+ LegacySchemaTables.mergeSchema(message.payload);
}
});
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/DefsTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
deleted file mode 100644
index 82a5dd1..0000000
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ /dev/null
@@ -1,622 +0,0 @@
-/*
- * 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.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.UTMetaData;
-import org.apache.cassandra.cql3.functions.Functions;
-import org.apache.cassandra.cql3.functions.UDAggregate;
-import org.apache.cassandra.cql3.functions.UDFunction;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.AsciiType;
-import org.apache.cassandra.db.marshal.UserType;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-/**
- * SCHEMA_{KEYSPACES, COLUMNFAMILIES, COLUMNS}_CF are used to store Keyspace/ColumnFamily attributes to make schema
- * load/distribution easy, it replaces old mechanism when local migrations where serialized, stored in system.Migrations
- * and used for schema distribution.
- */
-public class DefsTables
-{
- private static final Logger logger = LoggerFactory.getLogger(DefsTables.class);
-
- /**
- * Load keyspace definitions for the system keyspace (system.SCHEMA_KEYSPACES_TABLE)
- *
- * @return Collection of found keyspace definitions
- */
- public static Collection<KSMetaData> loadFromKeyspace()
- {
- List<Row> serializedSchema = SystemKeyspace.serializedSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE);
-
- List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
-
- for (Row row : serializedSchema)
- {
- if (Schema.invalidSchemaRow(row) || Schema.ignoredSchemaRow(row))
- continue;
-
- keyspaces.add(KSMetaData.fromSchema(row, serializedColumnFamilies(row.key), serializedUserTypes(row.key)));
- }
-
- return keyspaces;
- }
-
- private static Row serializedColumnFamilies(DecoratedKey ksNameKey)
- {
- ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE);
- return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
- SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE,
- System.currentTimeMillis())));
- }
-
- private static Row serializedUserTypes(DecoratedKey ksNameKey)
- {
- ColumnFamilyStore cfsStore = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
- return new Row(ksNameKey, cfsStore.getColumnFamily(QueryFilter.getIdentityFilter(ksNameKey,
- SystemKeyspace.SCHEMA_USER_TYPES_TABLE,
- System.currentTimeMillis())));
- }
-
- /**
- * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
- * (which also involves fs operations on add/drop ks/cf)
- *
- * @param mutations the schema changes to apply
- *
- * @throws ConfigurationException If one of metadata attributes has invalid value
- * @throws IOException If data was corrupted during transportation or failed to apply fs operations
- */
- public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
- {
- mergeSchemaInternal(mutations, true);
- Schema.instance.updateVersionAndAnnounce();
- }
-
- public static synchronized void mergeSchemaInternal(Collection<Mutation> mutations, boolean doFlush) throws IOException
- {
- // compare before/after schemas of the affected keyspaces only
- Set<String> keyspaces = new HashSet<>(mutations.size());
- for (Mutation mutation : mutations)
- keyspaces.add(ByteBufferUtil.string(mutation.key()));
-
- // current state of the schema
- Map<DecoratedKey, ColumnFamily> oldKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> oldColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> oldTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> oldFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> oldAggregates = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_AGGREGATES_TABLE, keyspaces);
-
- for (Mutation mutation : mutations)
- mutation.apply();
-
- if (doFlush)
- flushSchemaCFs();
-
- // with new data applied
- Map<DecoratedKey, ColumnFamily> newKeyspaces = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> newColumnFamilies = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> newTypes = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> newFunctions = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, keyspaces);
- Map<DecoratedKey, ColumnFamily> newAggregates = SystemKeyspace.getSchema(SystemKeyspace.SCHEMA_AGGREGATES_TABLE, keyspaces);
-
- Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
- mergeColumnFamilies(oldColumnFamilies, newColumnFamilies);
- mergeTypes(oldTypes, newTypes);
- mergeFunctions(oldFunctions, newFunctions);
- mergeAggregates(oldAggregates, newAggregates);
-
- // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
- for (String keyspaceToDrop : keyspacesToDrop)
- dropKeyspace(keyspaceToDrop);
- }
-
- private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
- {
- List<Row> created = new ArrayList<>();
- List<String> altered = new ArrayList<>();
- Set<String> dropped = new HashSet<>();
-
- /*
- * - we don't care about entriesOnlyOnLeft() or entriesInCommon(), because only the changes are of interest to us
- * - of all entriesOnlyOnRight(), we only care about ones that have live columns; it's possible to have a ColumnFamily
- * there that only has the top-level deletion, if:
- * a) a pushed DROP KEYSPACE change for a keyspace hadn't ever made it to this node in the first place
- * b) a pulled dropped keyspace that got dropped before it could find a way to this node
- * - of entriesDiffering(), we don't care about the scenario where both pre and post-values have zero live columns:
- * that means that a keyspace had been recreated and dropped, and the recreated keyspace had never found a way
- * to this node
- */
- MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
- if (entry.getValue().hasColumns())
- created.add(new Row(entry.getKey(), entry.getValue()));
-
- for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
- {
- String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
- ColumnFamily pre = entry.getValue().leftValue();
- ColumnFamily post = entry.getValue().rightValue();
-
- if (pre.hasColumns() && post.hasColumns())
- altered.add(keyspaceName);
- else if (pre.hasColumns())
- dropped.add(keyspaceName);
- else if (post.hasColumns()) // a (re)created keyspace
- created.add(new Row(entry.getKey(), post));
- }
-
- for (Row row : created)
- addKeyspace(KSMetaData.fromSchema(row, Collections.<CFMetaData>emptyList(), new UTMetaData()));
- for (String name : altered)
- updateKeyspace(name);
- return dropped;
- }
-
- // see the comments for mergeKeyspaces()
- private static void mergeColumnFamilies(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
- {
- List<CFMetaData> created = new ArrayList<>();
- List<CFMetaData> altered = new ArrayList<>();
- List<CFMetaData> dropped = new ArrayList<>();
-
- MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
- if (entry.getValue().hasColumns())
- created.addAll(KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), entry.getValue())).values());
-
- for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
- {
- String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
- ColumnFamily pre = entry.getValue().leftValue();
- ColumnFamily post = entry.getValue().rightValue();
-
- if (pre.hasColumns() && post.hasColumns())
- {
- MapDifference<String, CFMetaData> delta =
- Maps.difference(Schema.instance.getKSMetaData(keyspaceName).cfMetaData(),
- KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), post)));
-
- dropped.addAll(delta.entriesOnlyOnLeft().values());
- created.addAll(delta.entriesOnlyOnRight().values());
- Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<CFMetaData>, CFMetaData>()
- {
- public CFMetaData apply(MapDifference.ValueDifference<CFMetaData> pair)
- {
- return pair.rightValue();
- }
- }));
- }
- else if (pre.hasColumns())
- {
- dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).cfMetaData().values());
- }
- else if (post.hasColumns())
- {
- created.addAll(KSMetaData.deserializeColumnFamilies(new Row(entry.getKey(), post)).values());
- }
- }
-
- for (CFMetaData cfm : created)
- addColumnFamily(cfm);
- for (CFMetaData cfm : altered)
- updateColumnFamily(cfm.ksName, cfm.cfName);
- for (CFMetaData cfm : dropped)
- dropColumnFamily(cfm.ksName, cfm.cfName);
- }
-
- // see the comments for mergeKeyspaces()
- private static void mergeTypes(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
- {
- List<UserType> created = new ArrayList<>();
- List<UserType> altered = new ArrayList<>();
- List<UserType> dropped = new ArrayList<>();
-
- MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
- // New keyspace with types
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
- if (entry.getValue().hasColumns())
- created.addAll(UTMetaData.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
- for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
- {
- String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
-
- ColumnFamily pre = entry.getValue().leftValue();
- ColumnFamily post = entry.getValue().rightValue();
-
- if (pre.hasColumns() && post.hasColumns())
- {
- MapDifference<ByteBuffer, UserType> delta =
- Maps.difference(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes(),
- UTMetaData.fromSchema(new Row(entry.getKey(), post)));
-
- dropped.addAll(delta.entriesOnlyOnLeft().values());
- created.addAll(delta.entriesOnlyOnRight().values());
- Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UserType>, UserType>()
- {
- public UserType apply(MapDifference.ValueDifference<UserType> pair)
- {
- return pair.rightValue();
- }
- }));
- }
- else if (pre.hasColumns())
- {
- dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes().values());
- }
- else if (post.hasColumns())
- {
- created.addAll(UTMetaData.fromSchema(new Row(entry.getKey(), post)).values());
- }
- }
-
- for (UserType type : created)
- addType(type);
- for (UserType type : altered)
- updateType(type);
- for (UserType type : dropped)
- dropType(type);
- }
-
- // see the comments for mergeKeyspaces()
- private static void mergeFunctions(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
- {
- List<UDFunction> created = new ArrayList<>();
- List<UDFunction> altered = new ArrayList<>();
- List<UDFunction> dropped = new ArrayList<>();
-
- MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
- // New keyspace with functions
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
- if (entry.getValue().hasColumns())
- created.addAll(UDFunction.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
- for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
- {
- ColumnFamily pre = entry.getValue().leftValue();
- ColumnFamily post = entry.getValue().rightValue();
-
- if (pre.hasColumns() && post.hasColumns())
- {
- MapDifference<Composite, UDFunction> delta =
- Maps.difference(UDFunction.fromSchema(new Row(entry.getKey(), pre)),
- UDFunction.fromSchema(new Row(entry.getKey(), post)));
-
- dropped.addAll(delta.entriesOnlyOnLeft().values());
- created.addAll(delta.entriesOnlyOnRight().values());
- Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDFunction>, UDFunction>()
- {
- public UDFunction apply(MapDifference.ValueDifference<UDFunction> pair)
- {
- return pair.rightValue();
- }
- }));
- }
- else if (pre.hasColumns())
- {
- dropped.addAll(UDFunction.fromSchema(new Row(entry.getKey(), pre)).values());
- }
- else if (post.hasColumns())
- {
- created.addAll(UDFunction.fromSchema(new Row(entry.getKey(), post)).values());
- }
- }
-
- for (UDFunction udf : created)
- addFunction(udf);
- for (UDFunction udf : altered)
- updateFunction(udf);
- for (UDFunction udf : dropped)
- dropFunction(udf);
- }
-
- // see the comments for mergeKeyspaces()
- private static void mergeAggregates(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
- {
- List<UDAggregate> created = new ArrayList<>();
- List<UDAggregate> altered = new ArrayList<>();
- List<UDAggregate> dropped = new ArrayList<>();
-
- MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
-
- // New keyspace with functions
- for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
- if (entry.getValue().hasColumns())
- created.addAll(UDAggregate.fromSchema(new Row(entry.getKey(), entry.getValue())).values());
-
- for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
- {
- ColumnFamily pre = entry.getValue().leftValue();
- ColumnFamily post = entry.getValue().rightValue();
-
- if (pre.hasColumns() && post.hasColumns())
- {
- MapDifference<Composite, UDAggregate> delta =
- Maps.difference(UDAggregate.fromSchema(new Row(entry.getKey(), pre)),
- UDAggregate.fromSchema(new Row(entry.getKey(), post)));
-
- dropped.addAll(delta.entriesOnlyOnLeft().values());
- created.addAll(delta.entriesOnlyOnRight().values());
- Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDAggregate>, UDAggregate>()
- {
- public UDAggregate apply(MapDifference.ValueDifference<UDAggregate> pair)
- {
- return pair.rightValue();
- }
- }));
- }
- else if (pre.hasColumns())
- {
- dropped.addAll(UDAggregate.fromSchema(new Row(entry.getKey(), pre)).values());
- }
- else if (post.hasColumns())
- {
- created.addAll(UDAggregate.fromSchema(new Row(entry.getKey(), post)).values());
- }
- }
-
- for (UDAggregate udf : created)
- addAggregate(udf);
- for (UDAggregate udf : altered)
- updateAggregate(udf);
- for (UDAggregate udf : dropped)
- dropAggregate(udf);
- }
-
- private static void addKeyspace(KSMetaData ksm)
- {
- assert Schema.instance.getKSMetaData(ksm.name) == null;
- Schema.instance.load(ksm);
-
- Keyspace.open(ksm.name);
- MigrationManager.instance.notifyCreateKeyspace(ksm);
- }
-
- private static void addColumnFamily(CFMetaData cfm)
- {
- assert Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName) == null;
- KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
- ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(cfm)));
-
- logger.info("Loading {}", cfm);
-
- Schema.instance.load(cfm);
-
- // make sure it's init-ed w/ the old definitions first,
- // since we're going to call initCf on the new one manually
- Keyspace.open(cfm.ksName);
-
- Schema.instance.setKeyspaceDefinition(ksm);
- Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
- MigrationManager.instance.notifyCreateColumnFamily(cfm);
- }
-
- private static void addType(UserType ut)
- {
- KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
- assert ksm != null;
-
- logger.info("Loading {}", ut);
-
- ksm.userTypes.addType(ut);
-
- MigrationManager.instance.notifyCreateUserType(ut);
- }
-
- private static void addFunction(UDFunction udf)
- {
- logger.info("Loading {}", udf);
-
- Functions.addFunction(udf);
-
- MigrationManager.instance.notifyCreateFunction(udf);
- }
-
- private static void addAggregate(UDAggregate udf)
- {
- logger.info("Loading {}", udf);
-
- Functions.addFunction(udf);
-
- MigrationManager.instance.notifyCreateAggregate(udf);
- }
-
- private static void updateKeyspace(String ksName)
- {
- KSMetaData oldKsm = Schema.instance.getKSMetaData(ksName);
- assert oldKsm != null;
- KSMetaData newKsm = KSMetaData.cloneWith(oldKsm.reloadAttributes(), oldKsm.cfMetaData().values());
-
- Schema.instance.setKeyspaceDefinition(newKsm);
-
- Keyspace.open(ksName).createReplicationStrategy(newKsm);
- MigrationManager.instance.notifyUpdateKeyspace(newKsm);
- }
-
- private static void updateColumnFamily(String ksName, String cfName)
- {
- CFMetaData cfm = Schema.instance.getCFMetaData(ksName, cfName);
- assert cfm != null;
- cfm.reload();
-
- Keyspace keyspace = Keyspace.open(cfm.ksName);
- keyspace.getColumnFamilyStore(cfm.cfName).reload();
- MigrationManager.instance.notifyUpdateColumnFamily(cfm);
- }
-
- private static void updateType(UserType ut)
- {
- KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
- assert ksm != null;
-
- logger.info("Updating {}", ut);
-
- ksm.userTypes.addType(ut);
-
- MigrationManager.instance.notifyUpdateUserType(ut);
- }
-
- private static void updateFunction(UDFunction udf)
- {
- logger.info("Updating {}", udf);
-
- Functions.replaceFunction(udf);
-
- MigrationManager.instance.notifyUpdateFunction(udf);
- }
-
- private static void updateAggregate(UDAggregate udf)
- {
- logger.info("Updating {}", udf);
-
- Functions.replaceFunction(udf);
-
- MigrationManager.instance.notifyUpdateAggregate(udf);
- }
-
- private static void dropKeyspace(String ksName)
- {
- KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
- String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
-
- CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
-
- Keyspace keyspace = Keyspace.open(ksm.name);
-
- // remove all cfs from the keyspace instance.
- List<UUID> droppedCfs = new ArrayList<>();
- for (CFMetaData cfm : ksm.cfMetaData().values())
- {
- ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
-
- Schema.instance.purge(cfm);
-
- if (DatabaseDescriptor.isAutoSnapshot())
- cfs.snapshot(snapshotName);
- Keyspace.open(ksm.name).dropCf(cfm.cfId);
-
- droppedCfs.add(cfm.cfId);
- }
-
- // remove the keyspace from the static instances.
- Keyspace.clear(ksm.name);
- Schema.instance.clearKeyspaceDefinition(ksm);
-
- keyspace.writeOrder.awaitNewBarrier();
-
- // force a new segment in the CL
- CommitLog.instance.forceRecycleAllSegments(droppedCfs);
-
- MigrationManager.instance.notifyDropKeyspace(ksm);
- }
-
- private static void dropColumnFamily(String ksName, String cfName)
- {
- KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
- assert ksm != null;
- ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(cfName);
- assert cfs != null;
-
- // reinitialize the keyspace.
- CFMetaData cfm = ksm.cfMetaData().get(cfName);
-
- Schema.instance.purge(cfm);
- Schema.instance.setKeyspaceDefinition(makeNewKeyspaceDefinition(ksm, cfm));
-
- CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
-
- if (DatabaseDescriptor.isAutoSnapshot())
- cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
- Keyspace.open(ksm.name).dropCf(cfm.cfId);
- MigrationManager.instance.notifyDropColumnFamily(cfm);
-
- CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
- }
-
- private static void dropType(UserType ut)
- {
- KSMetaData ksm = Schema.instance.getKSMetaData(ut.keyspace);
- assert ksm != null;
-
- ksm.userTypes.removeType(ut);
-
- MigrationManager.instance.notifyDropUserType(ut);
- }
-
- private static void dropFunction(UDFunction udf)
- {
- logger.info("Drop {}", udf);
-
- // TODO: this is kind of broken as this remove all overloads of the function name
- Functions.removeFunction(udf.name(), udf.argTypes());
-
- MigrationManager.instance.notifyDropFunction(udf);
- }
-
- private static void dropAggregate(UDAggregate udf)
- {
- logger.info("Drop {}", udf);
-
- // TODO: this is kind of broken as this remove all overloads of the function name
- Functions.removeFunction(udf.name(), udf.argTypes());
-
- MigrationManager.instance.notifyDropAggregate(udf);
- }
-
- private static KSMetaData makeNewKeyspaceDefinition(KSMetaData ksm, CFMetaData toExclude)
- {
- // clone ksm but do not include the new def
- List<CFMetaData> newCfs = new ArrayList<>(ksm.cfMetaData().values());
- newCfs.remove(toExclude);
- assert newCfs.size() == ksm.cfMetaData().size() - 1;
- return KSMetaData.cloneWith(ksm, newCfs);
- }
-
- private static void flushSchemaCFs()
- {
- for (String cf : SystemKeyspace.ALL_SCHEMA_TABLES)
- SystemKeyspace.forceBlockingFlush(cf);
- }
-}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 081e01b..8c4477b 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -115,7 +115,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
new NamedThreadFactory("HintedHandoff", Thread.MIN_PRIORITY),
"internal");
- private final ColumnFamilyStore hintStore = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS_TABLE);
+ private final ColumnFamilyStore hintStore = Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS);
/**
* Returns a mutation representing a Hint to be sent to <code>targetId</code>
@@ -134,9 +134,9 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
UUID hintId = UUIDGen.getTimeUUID();
// serialize the hint with id and version as a composite column name
- CellName name = SystemKeyspace.HintsTable.comparator.makeCellName(hintId, MessagingService.current_version);
+ CellName name = SystemKeyspace.Hints.comparator.makeCellName(hintId, MessagingService.current_version);
ByteBuffer value = ByteBuffer.wrap(FBUtilities.serialize(mutation, Mutation.serializer, MessagingService.current_version));
- ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(SystemKeyspace.NAME, SystemKeyspace.HINTS_TABLE));
+ ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Schema.instance.getCFMetaData(SystemKeyspace.NAME, SystemKeyspace.HINTS));
cf.addColumn(name, value, now, ttl);
return new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(targetId), cf);
}
@@ -182,7 +182,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
private static void deleteHint(ByteBuffer tokenBytes, CellName columnName, long timestamp)
{
Mutation mutation = new Mutation(SystemKeyspace.NAME, tokenBytes);
- mutation.delete(SystemKeyspace.HINTS_TABLE, columnName, timestamp);
+ mutation.delete(SystemKeyspace.HINTS, columnName, timestamp);
mutation.applyUnsafe(); // don't bother with commitlog since we're going to flush as soon as we're done with delivery
}
@@ -207,7 +207,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
UUID hostId = StorageService.instance.getTokenMetadata().getHostId(endpoint);
ByteBuffer hostIdBytes = ByteBuffer.wrap(UUIDGen.decompose(hostId));
final Mutation mutation = new Mutation(SystemKeyspace.NAME, hostIdBytes);
- mutation.delete(SystemKeyspace.HINTS_TABLE, System.currentTimeMillis());
+ mutation.delete(SystemKeyspace.HINTS, System.currentTimeMillis());
// execute asynchronously to avoid blocking caller (which may be processing gossip)
Runnable runnable = new Runnable()
@@ -241,7 +241,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
try
{
logger.info("Truncating all stored hints.");
- Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS_TABLE).truncateBlocking();
+ Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.HINTS).truncateBlocking();
}
catch (Exception e)
{
@@ -375,7 +375,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
{
long now = System.currentTimeMillis();
QueryFilter filter = QueryFilter.getSliceFilter(epkey,
- SystemKeyspace.HINTS_TABLE,
+ SystemKeyspace.HINTS,
startColumn,
Composites.EMPTY,
false,
@@ -601,7 +601,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
try
{
RangeSliceCommand cmd = new RangeSliceCommand(SystemKeyspace.NAME,
- SystemKeyspace.HINTS_TABLE,
+ SystemKeyspace.HINTS,
System.currentTimeMillis(),
predicate,
range,
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/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 09fc338..b34d589 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -32,7 +32,6 @@ import java.util.concurrent.Future;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,6 +44,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
import org.apache.cassandra.db.filter.QueryFilter;
import org.apache.cassandra.db.index.SecondaryIndex;
import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.service.pager.QueryPagers;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index 4cf441e..2381f26 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -357,7 +357,7 @@ public class Memtable
// and BL data is strictly local, so we don't need to preserve tombstones for repair.
// If we have a data row + row level tombstone, then writing it is effectively an expensive no-op so we skip it.
// See CASSANDRA-4667.
- if (cfs.name.equals(SystemKeyspace.BATCHLOG_TABLE) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
+ if (cfs.name.equals(SystemKeyspace.BATCHLOG) && cfs.keyspace.getName().equals(SystemKeyspace.NAME))
continue;
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
index d4503ba..79753c1 100644
--- a/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
+++ b/src/java/org/apache/cassandra/db/MigrationRequestVerbHandler.java
@@ -26,6 +26,7 @@ import org.apache.cassandra.net.IVerbHandler;
import org.apache.cassandra.net.MessageIn;
import org.apache.cassandra.net.MessageOut;
import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.service.MigrationManager;
/**
@@ -40,7 +41,7 @@ public class MigrationRequestVerbHandler implements IVerbHandler
{
logger.debug("Received migration request from {}.", message.from);
MessageOut<Collection<Mutation>> response = new MessageOut<>(MessagingService.Verb.INTERNAL_RESPONSE,
- SystemKeyspace.serializeSchema(),
+ LegacySchemaTables.convertSchemaToMutations(),
MigrationManager.MigrationsSerializer.instance);
MessagingService.instance().sendReply(response, id, message.from);
}
[5/5] cassandra git commit: Isolate schema serializaton code
Posted by al...@apache.org.
Isolate schema serializaton code
patch by Aleksey Yeschenko; reviewed by Tyler Hobbs for CASSANDRA-8261
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3e9d345f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3e9d345f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3e9d345f
Branch: refs/heads/trunk
Commit: 3e9d345f0078922950157de4fd4c7992512b43b8
Parents: 32ac6af
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Dec 17 01:12:19 2014 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Dec 17 01:34:16 2014 +0300
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/cassandra/config/CFMetaData.java | 331 +---
.../cassandra/config/ColumnDefinition.java | 134 +-
.../cassandra/config/DatabaseDescriptor.java | 87 +-
.../org/apache/cassandra/config/KSMetaData.java | 155 +-
.../org/apache/cassandra/config/Schema.java | 248 ++-
.../cassandra/config/TriggerDefinition.java | 63 -
.../org/apache/cassandra/config/UTMetaData.java | 91 +-
.../cassandra/cql3/functions/Functions.java | 22 +-
.../cql3/functions/JavaSourceUDFFactory.java | 5 +-
.../cassandra/cql3/functions/UDAggregate.java | 206 +--
.../cassandra/cql3/functions/UDFunction.java | 193 +--
.../cassandra/cql3/functions/UDHelper.java | 12 +-
.../cql3/statements/CreateTableStatement.java | 24 +-
.../apache/cassandra/db/AtomicBTreeColumns.java | 3 +-
.../apache/cassandra/db/BatchlogManager.java | 14 +-
.../db/DefinitionsUpdateVerbHandler.java | 3 +-
.../org/apache/cassandra/db/DefsTables.java | 622 --------
.../cassandra/db/HintedHandOffManager.java | 16 +-
src/java/org/apache/cassandra/db/Keyspace.java | 2 +-
src/java/org/apache/cassandra/db/Memtable.java | 2 +-
.../db/MigrationRequestVerbHandler.java | 3 +-
.../org/apache/cassandra/db/SystemKeyspace.java | 514 ++----
.../hadoop/ColumnFamilyRecordReader.java | 28 +-
.../cassandra/hadoop/cql3/CqlRecordReader.java | 28 +-
.../cassandra/hadoop/cql3/CqlRecordWriter.java | 23 +-
.../hadoop/pig/AbstractCassandraStorage.java | 46 +-
.../apache/cassandra/hadoop/pig/CqlStorage.java | 26 +-
.../cassandra/io/sstable/CQLSSTableWriter.java | 10 +-
.../cassandra/schema/LegacySchemaTables.java | 1480 ++++++++++++++++++
.../cassandra/service/CassandraDaemon.java | 8 +-
.../apache/cassandra/service/ClientState.java | 3 +-
.../cassandra/service/MigrationManager.java | 127 +-
.../apache/cassandra/service/MigrationTask.java | 4 +-
.../apache/cassandra/service/StorageProxy.java | 2 +-
.../cassandra/service/StorageService.java | 4 +-
.../cassandra/thrift/ThriftConversion.java | 5 +-
.../org/apache/cassandra/tools/BulkLoader.java | 6 +-
.../apache/cassandra/tools/SSTableExport.java | 3 +-
.../apache/cassandra/tools/SSTableImport.java | 2 +-
.../cassandra/tools/SSTableLevelResetter.java | 3 +-
.../cassandra/tools/StandaloneScrubber.java | 3 +-
.../cassandra/tools/StandaloneSplitter.java | 4 +-
.../cassandra/tools/StandaloneUpgrader.java | 2 +-
.../apache/cassandra/config/CFMetaDataTest.java | 15 +-
.../config/DatabaseDescriptorTest.java | 7 +-
.../org/apache/cassandra/config/DefsTest.java | 564 -------
.../apache/cassandra/config/KSMetaDataTest.java | 6 +-
.../org/apache/cassandra/cql3/CQLTester.java | 4 +-
.../cassandra/db/BatchlogManagerTest.java | 4 +-
.../apache/cassandra/db/HintedHandOffTest.java | 8 +-
.../schema/LegacySchemaTablesTest.java | 568 +++++++
.../service/EmbeddedCassandraServiceTest.java | 2 +-
.../service/StorageServiceServerTest.java | 3 +-
54 files changed, 2792 insertions(+), 2957 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3571c1e..6f4cdec 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
3.0
+ * Modernize schema tables (CASSANDRA-8261)
* Support for user-defined aggregation functions (CASSANDRA-8053)
* Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419)
* Refactor SelectStatement, return IN results in natural order instead
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index eb78ec7..0730ba7 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -37,7 +37,6 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.cache.CachingOptions;
import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.cql3.statements.CFStatement;
import org.apache.cassandra.cql3.statements.CreateTableStatement;
import org.apache.cassandra.db.*;
@@ -50,14 +49,12 @@ import org.apache.cassandra.io.compress.CompressionParameters;
import org.apache.cassandra.io.compress.LZ4Compressor;
import org.apache.cassandra.io.sstable.format.Version;
import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.UUIDGen;
import org.github.jamm.Unmetered;
-import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
-import static org.apache.cassandra.utils.FBUtilities.json;
-
/**
* This class can be tricky to modify. Please read http://wiki.apache.org/cassandra/ConfigurationNotes for how to do so safely.
*/
@@ -221,7 +218,7 @@ public final class CFMetaData
public volatile CompressionParameters compressionParameters = new CompressionParameters(null);
// attribute setters that return the modified CFMetaData instance
- public CFMetaData comment(String prop) { comment = Strings.nullToEmpty(prop); return this;}
+ public CFMetaData comment(String prop) {comment = Strings.nullToEmpty(prop); return this;}
public CFMetaData readRepairChance(double prop) {readRepairChance = prop; return this;}
public CFMetaData dcLocalReadRepairChance(double prop) {dcLocalReadRepairChance = prop; return this;}
public CFMetaData gcGraceSeconds(int prop) {gcGraceSeconds = prop; return this;}
@@ -344,8 +341,8 @@ public final class CFMetaData
// Depends on parent's cache setting, turn on its index CF's cache.
// Row caching is never enabled; see CASSANDRA-5732
CachingOptions indexCaching = parent.getCaching().keyCache.isEnabled()
- ? CachingOptions.KEYS_ONLY
- : CachingOptions.NONE;
+ ? CachingOptions.KEYS_ONLY
+ : CachingOptions.NONE;
return new CFMetaData(parent.ksName, parent.indexColumnFamilyName(info), ColumnFamilyType.Standard, indexComparator, parent.cfId)
.keyValidator(info.type)
@@ -386,7 +383,8 @@ public final class CFMetaData
return copyOpts(new CFMetaData(ksName, cfName, cfType, comparator, newCfId), this);
}
- static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
+ @VisibleForTesting
+ public static CFMetaData copyOpts(CFMetaData newCFMD, CFMetaData oldCFMD)
{
List<ColumnDefinition> clonedColumns = new ArrayList<>(oldCFMD.allColumns().size());
for (ColumnDefinition cd : oldCFMD.allColumns())
@@ -449,6 +447,11 @@ public final class CFMetaData
return cfName.contains(".");
}
+ public Map<ByteBuffer, ColumnDefinition> getColumnMetadata()
+ {
+ return columnMetadata;
+ }
+
/**
*
* @return The name of the parent cf if this is a seconday index
@@ -723,14 +726,9 @@ public final class CFMetaData
public void reload()
{
- Row cfDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, ksName, cfName);
-
- if (cfDefRow.cf == null || !cfDefRow.cf.hasColumns())
- throw new RuntimeException(String.format("%s not found in the schema definitions keyspace.", ksName + ":" + cfName));
-
try
{
- apply(fromSchema(cfDefRow));
+ apply(LegacySchemaTables.createTableFromName(ksName, cfName));
}
catch (ConfigurationException e)
{
@@ -739,13 +737,12 @@ public final class CFMetaData
}
/**
- * Updates CFMetaData in-place to match cf_def
- *
- * *Note*: This method left package-private only for DefsTest, don't use directly!
+ * Updates CFMetaData in-place to match cfm
*
* @throws ConfigurationException if ks/cf names or cf ids didn't match
*/
- void apply(CFMetaData cfm) throws ConfigurationException
+ @VisibleForTesting
+ public void apply(CFMetaData cfm) throws ConfigurationException
{
logger.debug("applying {} to {}", cfm, this);
@@ -1116,89 +1113,6 @@ public final class CFMetaData
"interval (%d).", maxIndexInterval, minIndexInterval));
}
- /**
- * Create schema mutations to update this metadata to provided new state.
- *
- * @param newState The new metadata (for the same CF)
- * @param modificationTimestamp Timestamp to use for mutation
- * @param fromThrift whether the newState comes from thrift
- *
- * @return Difference between attributes in form of schema mutation
- */
- public Mutation toSchemaUpdate(CFMetaData newState, long modificationTimestamp, boolean fromThrift)
- {
- Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
-
- newState.toSchemaNoColumnsNoTriggers(mutation, modificationTimestamp);
-
- MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(columnMetadata, newState.columnMetadata);
-
- // columns that are no longer needed
- for (ColumnDefinition cd : columnDiff.entriesOnlyOnLeft().values())
- {
- // Thrift only knows about the REGULAR ColumnDefinition type, so don't consider other type
- // are being deleted just because they are not here.
- if (fromThrift && cd.kind != ColumnDefinition.Kind.REGULAR)
- continue;
-
- cd.deleteFromSchema(mutation, modificationTimestamp);
- }
-
- // newly added columns
- for (ColumnDefinition cd : columnDiff.entriesOnlyOnRight().values())
- cd.toSchema(mutation, modificationTimestamp);
-
- // old columns with updated attributes
- for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
- {
- ColumnDefinition cd = newState.columnMetadata.get(name);
- cd.toSchema(mutation, modificationTimestamp);
- }
-
- MapDifference<String, TriggerDefinition> triggerDiff = Maps.difference(triggers, newState.triggers);
-
- // dropped triggers
- for (TriggerDefinition td : triggerDiff.entriesOnlyOnLeft().values())
- td.deleteFromSchema(mutation, cfName, modificationTimestamp);
-
- // newly created triggers
- for (TriggerDefinition td : triggerDiff.entriesOnlyOnRight().values())
- td.toSchema(mutation, cfName, modificationTimestamp);
-
- return mutation;
- }
-
- /**
- * Remove all CF attributes from schema
- *
- * @param timestamp Timestamp to use
- *
- * @return Mutation to use to completely remove cf from schema
- */
- public Mutation dropFromSchema(long timestamp)
- {
- Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnFamiliesTable);
- int ldt = (int) (System.currentTimeMillis() / 1000);
-
- Composite prefix = SystemKeyspace.SchemaColumnFamiliesTable.comparator.make(cfName);
- cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
- for (ColumnDefinition cd : allColumns())
- cd.deleteFromSchema(mutation, timestamp);
-
- for (TriggerDefinition td : triggers.values())
- td.deleteFromSchema(mutation, cfName, timestamp);
-
- for (String indexName : Keyspace.open(this.ksName).getColumnFamilyStore(this.cfName).getBuiltIndexes())
- {
- ColumnFamily indexCf = mutation.addOrGet(SystemKeyspace.BuiltIndexesTable);
- indexCf.addTombstone(indexCf.getComparator().makeCellName(indexName), ldt, timestamp);
- }
-
- return mutation;
- }
-
public boolean isPurged()
{
return isPurged;
@@ -1209,215 +1123,6 @@ public final class CFMetaData
isPurged = true;
}
- public void toSchema(Mutation mutation, long timestamp)
- {
- toSchemaNoColumnsNoTriggers(mutation, timestamp);
-
- for (TriggerDefinition td : triggers.values())
- td.toSchema(mutation, cfName, timestamp);
-
- for (ColumnDefinition cd : allColumns())
- cd.toSchema(mutation, timestamp);
- }
-
- private void toSchemaNoColumnsNoTriggers(Mutation mutation, long timestamp)
- {
- // For property that can be null (and can be changed), we insert tombstones, to make sure
- // we don't keep a property the user has removed
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnFamiliesTable);
- Composite prefix = SystemKeyspace.SchemaColumnFamiliesTable.comparator.make(cfName);
- CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
- adder.add("cf_id", cfId);
- adder.add("type", cfType.toString());
-
- if (isSuper())
- {
- // We need to continue saving the comparator and subcomparator separatly, otherwise
- // we won't know at deserialization if the subcomparator should be taken into account
- // TODO: we should implement an on-start migration if we want to get rid of that.
- adder.add("comparator", comparator.subtype(0).toString());
- adder.add("subcomparator", comparator.subtype(1).toString());
- }
- else
- {
- adder.add("comparator", comparator.toString());
- }
-
- adder.add("comment", comment);
- adder.add("read_repair_chance", readRepairChance);
- adder.add("local_read_repair_chance", dcLocalReadRepairChance);
- adder.add("gc_grace_seconds", gcGraceSeconds);
- adder.add("default_validator", defaultValidator.toString());
- adder.add("key_validator", keyValidator.toString());
- adder.add("min_compaction_threshold", minCompactionThreshold);
- adder.add("max_compaction_threshold", maxCompactionThreshold);
- adder.add("bloom_filter_fp_chance", getBloomFilterFpChance());
- adder.add("memtable_flush_period_in_ms", memtableFlushPeriod);
- adder.add("caching", caching.toString());
- adder.add("default_time_to_live", defaultTimeToLive);
- adder.add("compaction_strategy_class", compactionStrategyClass.getName());
- adder.add("compression_parameters", json(compressionParameters.asThriftOptions()));
- adder.add("compaction_strategy_options", json(compactionStrategyOptions));
- adder.add("min_index_interval", minIndexInterval);
- adder.add("max_index_interval", maxIndexInterval);
- adder.add("speculative_retry", speculativeRetry.toString());
-
- for (Map.Entry<ColumnIdentifier, Long> entry : droppedColumns.entrySet())
- adder.addMapEntry("dropped_columns", entry.getKey().toString(), entry.getValue());
-
- adder.add("is_dense", isDense);
- }
-
- @VisibleForTesting
- public static CFMetaData fromSchemaNoTriggers(UntypedResultSet.Row result, UntypedResultSet serializedColumnDefinitions)
- {
- try
- {
- String ksName = result.getString("keyspace_name");
- String cfName = result.getString("columnfamily_name");
-
- AbstractType<?> rawComparator = TypeParser.parse(result.getString("comparator"));
- AbstractType<?> subComparator = result.has("subcomparator") ? TypeParser.parse(result.getString("subcomparator")) : null;
- ColumnFamilyType cfType = ColumnFamilyType.valueOf(result.getString("type"));
-
- AbstractType<?> fullRawComparator = makeRawAbstractType(rawComparator, subComparator);
-
- List<ColumnDefinition> columnDefs = ColumnDefinition.fromSchema(serializedColumnDefinitions,
- ksName,
- cfName,
- fullRawComparator,
- cfType == ColumnFamilyType.Super);
-
- boolean isDense = result.has("is_dense")
- ? result.getBoolean("is_dense")
- : calculateIsDense(fullRawComparator, columnDefs);
-
- CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, isDense);
-
- // if we are upgrading, we use id generated from names initially
- UUID cfId = result.has("cf_id")
- ? result.getUUID("cf_id")
- : generateLegacyCfId(ksName, cfName);
-
- CFMetaData cfm = new CFMetaData(ksName, cfName, cfType, comparator, cfId);
- cfm.isDense(isDense);
-
- cfm.readRepairChance(result.getDouble("read_repair_chance"));
- cfm.dcLocalReadRepairChance(result.getDouble("local_read_repair_chance"));
- cfm.gcGraceSeconds(result.getInt("gc_grace_seconds"));
- cfm.defaultValidator(TypeParser.parse(result.getString("default_validator")));
- cfm.keyValidator(TypeParser.parse(result.getString("key_validator")));
- cfm.minCompactionThreshold(result.getInt("min_compaction_threshold"));
- cfm.maxCompactionThreshold(result.getInt("max_compaction_threshold"));
- if (result.has("comment"))
- cfm.comment(result.getString("comment"));
- if (result.has("memtable_flush_period_in_ms"))
- cfm.memtableFlushPeriod(result.getInt("memtable_flush_period_in_ms"));
- cfm.caching(CachingOptions.fromString(result.getString("caching")));
- if (result.has("default_time_to_live"))
- cfm.defaultTimeToLive(result.getInt("default_time_to_live"));
- if (result.has("speculative_retry"))
- cfm.speculativeRetry(SpeculativeRetry.fromString(result.getString("speculative_retry")));
- cfm.compactionStrategyClass(createCompactionStrategy(result.getString("compaction_strategy_class")));
- cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
- cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
-
- if (result.has("min_index_interval"))
- cfm.minIndexInterval(result.getInt("min_index_interval"));
-
- if (result.has("max_index_interval"))
- cfm.maxIndexInterval(result.getInt("max_index_interval"));
-
- if (result.has("bloom_filter_fp_chance"))
- cfm.bloomFilterFpChance(result.getDouble("bloom_filter_fp_chance"));
- else
- cfm.bloomFilterFpChance(cfm.getBloomFilterFpChance());
-
- if (result.has("dropped_columns"))
- cfm.droppedColumns(convertDroppedColumns(result.getMap("dropped_columns", UTF8Type.instance, LongType.instance)));
-
- for (ColumnDefinition cd : columnDefs)
- cfm.addOrReplaceColumnDefinition(cd);
-
- return cfm.rebuild();
- }
- catch (SyntaxException | ConfigurationException e)
- {
- throw new RuntimeException(e);
- }
- }
-
- public void addColumnMetadataFromAliases(List<ByteBuffer> aliases, AbstractType<?> comparator, ColumnDefinition.Kind kind)
- {
- if (comparator instanceof CompositeType)
- {
- CompositeType ct = (CompositeType)comparator;
- for (int i = 0; i < aliases.size(); ++i)
- {
- if (aliases.get(i) != null)
- {
- addOrReplaceColumnDefinition(new ColumnDefinition(this, aliases.get(i), ct.types.get(i), i, kind));
- }
- }
- }
- else
- {
- assert aliases.size() <= 1;
- if (!aliases.isEmpty() && aliases.get(0) != null)
- addOrReplaceColumnDefinition(new ColumnDefinition(this, aliases.get(0), comparator, null, kind));
- }
- }
-
- /**
- * Deserialize CF metadata from low-level representation
- *
- * @return Metadata deserialized from schema
- */
- public static CFMetaData fromSchema(UntypedResultSet.Row result)
- {
- String ksName = result.getString("keyspace_name");
- String cfName = result.getString("columnfamily_name");
-
- Row serializedColumns = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_COLUMNS_TABLE, ksName, cfName);
- CFMetaData cfm = fromSchemaNoTriggers(result, ColumnDefinition.resultify(serializedColumns));
-
- Row serializedTriggers = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_TRIGGERS_TABLE, ksName, cfName);
- addTriggerDefinitionsFromSchema(cfm, serializedTriggers);
-
- return cfm;
- }
-
- private static CFMetaData fromSchema(Row row)
- {
- UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row).one();
- return fromSchema(result);
- }
-
- private static Map<ColumnIdentifier, Long> convertDroppedColumns(Map<String, Long> raw)
- {
- Map<ColumnIdentifier, Long> converted = Maps.newHashMap();
- for (Map.Entry<String, Long> entry : raw.entrySet())
- converted.put(new ColumnIdentifier(entry.getKey(), true), entry.getValue());
- return converted;
- }
-
- /**
- * Convert current metadata into schema mutation
- *
- * @param timestamp Timestamp to use
- *
- * @return Low-level representation of the CF
- *
- * @throws ConfigurationException if any of the attributes didn't pass validation
- */
- public Mutation toSchema(long timestamp) throws ConfigurationException
- {
- Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(ksName));
- toSchema(mutation, timestamp);
- return mutation;
- }
-
// The comparator to validate the definition name.
public AbstractType<?> getColumnDefinitionComparator(ColumnDefinition def)
@@ -1474,12 +1179,6 @@ public final class CFMetaData
return columnMetadata.remove(def.name.bytes) != null;
}
- private static void addTriggerDefinitionsFromSchema(CFMetaData cfDef, Row serializedTriggerDefinitions)
- {
- for (TriggerDefinition td : TriggerDefinition.fromSchema(serializedTriggerDefinitions))
- cfDef.triggers.put(td.name, td);
- }
-
public void addTriggerDefinition(TriggerDefinition def) throws InvalidRequestException
{
if (containsTriggerDefinition(def))
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/ColumnDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java
index 354a6f1..1cc7f1d 100644
--- a/src/java/org/apache/cassandra/config/ColumnDefinition.java
+++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java
@@ -26,25 +26,11 @@ import com.google.common.base.Objects;
import com.google.common.collect.Lists;
import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
import org.apache.cassandra.db.marshal.*;
import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.apache.cassandra.utils.FBUtilities.json;
public class ColumnDefinition extends ColumnSpecification
{
- // system.schema_columns column names
- private static final String COLUMN_NAME = "column_name";
- private static final String TYPE = "validator";
- private static final String INDEX_TYPE = "index_type";
- private static final String INDEX_OPTIONS = "index_options";
- private static final String INDEX_NAME = "index_name";
- private static final String COMPONENT_INDEX = "component_index";
- private static final String KIND = "type";
-
/*
* The type of CQL3 column this definition represents.
* There is 3 main type of CQL3 columns: those parts of the partition key,
@@ -62,20 +48,7 @@ public class ColumnDefinition extends ColumnSpecification
CLUSTERING_COLUMN,
REGULAR,
STATIC,
- COMPACT_VALUE;
-
- public String serialize()
- {
- // For backward compatibility we need to special case CLUSTERING_COLUMN
- return this == CLUSTERING_COLUMN ? "clustering_key" : this.toString().toLowerCase();
- }
-
- public static Kind deserialize(String value)
- {
- if (value.equalsIgnoreCase("clustering_key"))
- return CLUSTERING_COLUMN;
- return Enum.valueOf(Kind.class, value.toUpperCase());
- }
+ COMPACT_VALUE
}
public final Kind kind;
@@ -266,36 +239,6 @@ public class ColumnDefinition extends ColumnSpecification
return kind == Kind.REGULAR || kind == Kind.STATIC;
}
- /**
- * Drop specified column from the schema using given mutation.
- *
- * @param mutation The schema mutation
- * @param timestamp The timestamp to use for column modification
- */
- public void deleteFromSchema(Mutation mutation, long timestamp)
- {
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnsTable);
- int ldt = (int) (System.currentTimeMillis() / 1000);
-
- // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
- Composite prefix = SystemKeyspace.SchemaColumnsTable.comparator.make(cfName, name.toString());
- cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
- }
-
- public void toSchema(Mutation mutation, long timestamp)
- {
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaColumnsTable);
- Composite prefix = SystemKeyspace.SchemaColumnsTable.comparator.make(cfName, name.toString());
- CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
- adder.add(TYPE, type.toString());
- adder.add(INDEX_TYPE, indexType == null ? null : indexType.toString());
- adder.add(INDEX_OPTIONS, json(indexOptions));
- adder.add(INDEX_NAME, indexName);
- adder.add(COMPONENT_INDEX, componentIndex);
- adder.add(KIND, kind.serialize());
- }
-
public ColumnDefinition apply(ColumnDefinition def) throws ConfigurationException
{
assert kind == def.kind && Objects.equal(componentIndex, def.componentIndex);
@@ -323,81 +266,6 @@ public class ColumnDefinition extends ColumnSpecification
kind);
}
- public static UntypedResultSet resultify(Row serializedColumns)
- {
- String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.SCHEMA_COLUMNS_TABLE);
- return QueryProcessor.resultify(query, serializedColumns);
- }
-
- /**
- * Deserialize columns from storage-level representation
- *
- * @param serializedColumns storage-level partition containing the column definitions
- * @return the list of processed ColumnDefinitions
- */
- public static List<ColumnDefinition> fromSchema(UntypedResultSet serializedColumns, String ksName, String cfName, AbstractType<?> rawComparator, boolean isSuper)
- {
- List<ColumnDefinition> cds = new ArrayList<>();
- for (UntypedResultSet.Row row : serializedColumns)
- {
- Kind kind = row.has(KIND)
- ? Kind.deserialize(row.getString(KIND))
- : Kind.REGULAR;
-
- Integer componentIndex = null;
- if (row.has(COMPONENT_INDEX))
- componentIndex = row.getInt(COMPONENT_INDEX);
- else if (kind == Kind.CLUSTERING_COLUMN && isSuper)
- componentIndex = 1; // A ColumnDefinition for super columns applies to the column component
-
- // Note: we save the column name as string, but we should not assume that it is an UTF8 name, we
- // we need to use the comparator fromString method
- AbstractType<?> comparator = getComponentComparator(rawComparator, componentIndex, kind);
- ColumnIdentifier name = new ColumnIdentifier(comparator.fromString(row.getString(COLUMN_NAME)), comparator);
-
- AbstractType<?> validator;
- try
- {
- validator = TypeParser.parse(row.getString(TYPE));
- }
- catch (RequestValidationException e)
- {
- throw new RuntimeException(e);
- }
-
- IndexType indexType = null;
- if (row.has(INDEX_TYPE))
- indexType = IndexType.valueOf(row.getString(INDEX_TYPE));
-
- Map<String, String> indexOptions = null;
- if (row.has(INDEX_OPTIONS))
- indexOptions = FBUtilities.fromJsonMap(row.getString(INDEX_OPTIONS));
-
- String indexName = null;
- if (row.has(INDEX_NAME))
- indexName = row.getString(INDEX_NAME);
-
- cds.add(new ColumnDefinition(ksName, cfName, name, validator, indexType, indexOptions, indexName, componentIndex, kind));
- }
-
- return cds;
- }
-
- public static AbstractType<?> getComponentComparator(AbstractType<?> rawComparator, Integer componentIndex, ColumnDefinition.Kind kind)
- {
- switch (kind)
- {
- case REGULAR:
- if (componentIndex == null || (componentIndex == 0 && !(rawComparator instanceof CompositeType)))
- return rawComparator;
-
- return ((CompositeType)rawComparator).types.get(componentIndex);
- default:
- // CQL3 column names are UTF8
- return UTF8Type.instance;
- }
- }
-
public String getIndexName()
{
return indexName;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index a0e84f9..f2897ee 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -18,49 +18,29 @@
package org.apache.cassandra.config;
import java.io.File;
-import java.io.FileFilter;
import java.io.IOException;
-import java.net.InetAddress;
-import java.net.NetworkInterface;
-import java.net.SocketException;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Enumeration;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.net.*;
+import java.util.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet;
import com.google.common.primitives.Longs;
-import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.auth.AllowAllAuthenticator;
-import org.apache.cassandra.auth.AllowAllAuthorizer;
-import org.apache.cassandra.auth.AllowAllInternodeAuthenticator;
-import org.apache.cassandra.auth.IAuthenticator;
-import org.apache.cassandra.auth.IAuthorizer;
-import org.apache.cassandra.auth.IInternodeAuthenticator;
+
+import org.apache.cassandra.auth.*;
import org.apache.cassandra.config.Config.RequestSchedulerId;
import org.apache.cassandra.config.EncryptionOptions.ClientEncryptionOptions;
import org.apache.cassandra.config.EncryptionOptions.ServerEncryptionOptions;
import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DefsTables;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.IAllocator;
-import org.apache.cassandra.locator.DynamicEndpointSnitch;
-import org.apache.cassandra.locator.EndpointSnitchInfo;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.SeedProvider;
+import org.apache.cassandra.locator.*;
import org.apache.cassandra.net.MessagingService;
import org.apache.cassandra.scheduler.IRequestScheduler;
import org.apache.cassandra.scheduler.NoScheduler;
@@ -69,10 +49,7 @@ import org.apache.cassandra.thrift.ThriftServer;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.memory.HeapPool;
-import org.apache.cassandra.utils.memory.NativePool;
-import org.apache.cassandra.utils.memory.MemtablePool;
-import org.apache.cassandra.utils.memory.SlabPool;
+import org.apache.cassandra.utils.memory.*;
public class DatabaseDescriptor
{
@@ -585,9 +562,6 @@ public class DatabaseDescriptor
conf.server_encryption_options = conf.encryption_options;
}
- // hardcoded system keyspace
- Schema.instance.load(SystemKeyspace.definition());
-
// load the seeds for node contact points
if (conf.seed_provider == null)
{
@@ -620,53 +594,6 @@ public class DatabaseDescriptor
return conf.dynamic_snitch ? new DynamicEndpointSnitch(snitch) : snitch;
}
- /** load keyspace (keyspace) definitions, but do not initialize the keyspace instances. */
- public static void loadSchemas()
- {
- ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(SystemKeyspace.SCHEMA_KEYSPACES_TABLE);
-
- // if keyspace with definitions is empty try loading the old way
- if (schemaCFS.estimateKeys() == 0)
- {
- logger.info("Couldn't detect any schema definitions in local storage.");
- // peek around the data directories to see if anything is there.
- if (hasExistingNoSystemTables())
- logger.info("Found keyspace data in data directories. Consider using cqlsh to define your schema.");
- else
- logger.info("To create keyspaces and column families, see 'help create' in cqlsh.");
- }
- else
- {
- Schema.instance.load(DefsTables.loadFromKeyspace());
- }
-
- Schema.instance.updateVersion();
- }
-
- private static boolean hasExistingNoSystemTables()
- {
- for (String dataDir : getAllDataFileLocations())
- {
- File dataPath = new File(dataDir);
- if (dataPath.exists() && dataPath.isDirectory())
- {
- // see if there are other directories present.
- int dirCount = dataPath.listFiles(new FileFilter()
- {
- public boolean accept(File pathname)
- {
- return pathname.isDirectory() && !pathname.getName().equals(SystemKeyspace.NAME);
- }
- }).length;
-
- if (dirCount > 0)
- return true;
- }
- }
-
- return false;
- }
-
public static IAuthenticator getAuthenticator()
{
return authenticator;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/KSMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/KSMetaData.java b/src/java/org/apache/cassandra/config/KSMetaData.java
index e5576ad..1537aae 100644
--- a/src/java/org/apache/cassandra/config/KSMetaData.java
+++ b/src/java/org/apache/cassandra/config/KSMetaData.java
@@ -21,15 +21,10 @@ import java.util.*;
import com.google.common.base.Objects;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.locator.*;
import org.apache.cassandra.service.StorageService;
-import static org.apache.cassandra.utils.FBUtilities.*;
-
public final class KSMetaData
{
public final String name;
@@ -43,18 +38,26 @@ public final class KSMetaData
public KSMetaData(String name,
Class<? extends AbstractReplicationStrategy> strategyClass,
Map<String, String> strategyOptions,
+ boolean durableWrites)
+ {
+ this(name, strategyClass, strategyOptions, durableWrites, Collections.<CFMetaData>emptyList(), new UTMetaData());
+ }
+
+ public KSMetaData(String name,
+ Class<? extends AbstractReplicationStrategy> strategyClass,
+ Map<String, String> strategyOptions,
boolean durableWrites,
Iterable<CFMetaData> cfDefs)
{
this(name, strategyClass, strategyOptions, durableWrites, cfDefs, new UTMetaData());
}
- KSMetaData(String name,
- Class<? extends AbstractReplicationStrategy> strategyClass,
- Map<String, String> strategyOptions,
- boolean durableWrites,
- Iterable<CFMetaData> cfDefs,
- UTMetaData userTypes)
+ private KSMetaData(String name,
+ Class<? extends AbstractReplicationStrategy> strategyClass,
+ Map<String, String> strategyOptions,
+ boolean durableWrites,
+ Iterable<CFMetaData> cfDefs,
+ UTMetaData userTypes)
{
this.name = name;
this.strategyClass = strategyClass == null ? NetworkTopologyStrategy.class : strategyClass;
@@ -82,9 +85,27 @@ public final class KSMetaData
return new KSMetaData(name, strategyClass, options, durablesWrites, cfDefs, new UTMetaData());
}
- public static KSMetaData cloneWith(KSMetaData ksm, Iterable<CFMetaData> cfDefs)
+ public KSMetaData cloneWithTableRemoved(CFMetaData table)
+ {
+ // clone ksm but do not include the new table
+ List<CFMetaData> newTables = new ArrayList<>(cfMetaData().values());
+ newTables.remove(table);
+ assert newTables.size() == cfMetaData().size() - 1;
+ return cloneWith(newTables, userTypes);
+ }
+
+ public KSMetaData cloneWithTableAdded(CFMetaData table)
+ {
+ // clone ksm but include the new table
+ List<CFMetaData> newTables = new ArrayList<>(cfMetaData().values());
+ newTables.add(table);
+ assert newTables.size() == cfMetaData().size() + 1;
+ return cloneWith(newTables, userTypes);
+ }
+
+ public KSMetaData cloneWith(Iterable<CFMetaData> tables, UTMetaData types)
{
- return new KSMetaData(ksm.name, ksm.strategyClass, ksm.strategyOptions, ksm.durableWrites, cfDefs, ksm.userTypes);
+ return new KSMetaData(name, strategyClass, strategyOptions, durableWrites, tables, types);
}
public static KSMetaData testMetadata(String name, Class<? extends AbstractReplicationStrategy> strategyClass, Map<String, String> strategyOptions, CFMetaData... cfDefs)
@@ -145,11 +166,6 @@ public final class KSMetaData
return Collections.singletonMap("replication_factor", rf.toString());
}
- public Mutation toSchemaUpdate(KSMetaData newState, long modificationTimestamp)
- {
- return newState.toSchema(modificationTimestamp);
- }
-
public KSMetaData validate() throws ConfigurationException
{
if (!CFMetaData.isNameValid(name))
@@ -165,107 +181,4 @@ public final class KSMetaData
return this;
}
-
- public KSMetaData reloadAttributes()
- {
- Row ksDefRow = SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, name);
-
- if (ksDefRow.cf == null)
- throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", name, SystemKeyspace.SCHEMA_KEYSPACES_TABLE));
-
- return fromSchema(ksDefRow, Collections.<CFMetaData>emptyList(), userTypes);
- }
-
- public Mutation dropFromSchema(long timestamp)
- {
- Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(name));
-
- mutation.delete(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, timestamp);
- mutation.delete(SystemKeyspace.SCHEMA_COLUMNFAMILIES_TABLE, timestamp);
- mutation.delete(SystemKeyspace.SCHEMA_COLUMNS_TABLE, timestamp);
- mutation.delete(SystemKeyspace.SCHEMA_TRIGGERS_TABLE, timestamp);
- mutation.delete(SystemKeyspace.SCHEMA_USER_TYPES_TABLE, timestamp);
- mutation.delete(SystemKeyspace.SCHEMA_FUNCTIONS_TABLE, timestamp);
- mutation.delete(SystemKeyspace.SCHEMA_AGGREGATES_TABLE, timestamp);
- mutation.delete(SystemKeyspace.BUILT_INDEXES_TABLE, timestamp);
-
- return mutation;
- }
-
- public Mutation toSchema(long timestamp)
- {
- Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(name));
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SchemaKeyspacesTable);
- CFRowAdder adder = new CFRowAdder(cf, SystemKeyspace.SchemaKeyspacesTable.comparator.builder().build(), timestamp);
-
- adder.add("durable_writes", durableWrites);
- adder.add("strategy_class", strategyClass.getName());
- adder.add("strategy_options", json(strategyOptions));
-
- for (CFMetaData cfm : cfMetaData.values())
- cfm.toSchema(mutation, timestamp);
-
- userTypes.toSchema(mutation, timestamp);
- return mutation;
- }
-
- /**
- * Deserialize only Keyspace attributes without nested ColumnFamilies
- *
- * @param row Keyspace attributes in serialized form
- *
- * @return deserialized keyspace without cf_defs
- */
- public static KSMetaData fromSchema(Row row, Iterable<CFMetaData> cfms, UTMetaData userTypes)
- {
- UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_keyspaces", row).one();
- try
- {
- return new KSMetaData(result.getString("keyspace_name"),
- AbstractReplicationStrategy.getClass(result.getString("strategy_class")),
- fromJsonMap(result.getString("strategy_options")),
- result.getBoolean("durable_writes"),
- cfms,
- userTypes);
- }
- catch (ConfigurationException e)
- {
- throw new RuntimeException(e);
- }
- }
-
- /**
- * Deserialize Keyspace with nested ColumnFamilies
- *
- * @param serializedKs Keyspace in serialized form
- * @param serializedCFs Collection of the serialized ColumnFamilies
- *
- * @return deserialized keyspace with cf_defs
- */
- public static KSMetaData fromSchema(Row serializedKs, Row serializedCFs, Row serializedUserTypes)
- {
- Map<String, CFMetaData> cfs = deserializeColumnFamilies(serializedCFs);
- UTMetaData userTypes = new UTMetaData(UTMetaData.fromSchema(serializedUserTypes));
- return fromSchema(serializedKs, cfs.values(), userTypes);
- }
-
- /**
- * Deserialize ColumnFamilies from low-level schema representation, all of them belong to the same keyspace
- *
- * @return map containing name of the ColumnFamily and it's metadata for faster lookup
- */
- public static Map<String, CFMetaData> deserializeColumnFamilies(Row row)
- {
- if (row.cf == null)
- return Collections.emptyMap();
-
- UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row);
- Map<String, CFMetaData> cfms = new HashMap<>(results.size());
- for (UntypedResultSet.Row result : results)
- {
- CFMetaData cfm = CFMetaData.fromSchema(result);
- cfms.put(cfm.cfName, cfm);
- }
- return cfms;
- }
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/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 43cc6b5..21244ab 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -17,7 +17,6 @@
*/
package org.apache.cassandra.config;
-import java.nio.charset.CharacterCodingException;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.*;
@@ -27,13 +26,18 @@ import com.google.common.collect.Sets;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.cassandra.cql3.functions.Functions;
+import org.apache.cassandra.cql3.functions.UDAggregate;
+import org.apache.cassandra.cql3.functions.UDFunction;
import org.apache.cassandra.db.*;
import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.UserType;
import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.service.MigrationManager;
import org.apache.cassandra.utils.ConcurrentBiMap;
-import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.Pair;
import org.cliffc.high_scale_lib.NonBlockingHashMap;
@@ -78,10 +82,20 @@ public class Schema
}
/**
- * Initialize empty schema object
+ * Initialize empty schema object and load the hardcoded system tables
*/
public Schema()
- {}
+ {
+ load(SystemKeyspace.definition());
+ }
+
+ /** load keyspace (keyspace) definitions, but do not initialize the keyspace instances. */
+ public Schema loadFromDisk()
+ {
+ load(LegacySchemaTables.readSchemaFromSystemTables());
+ updateVersion();
+ return this;
+ }
/**
* Load up non-system keyspaces
@@ -350,28 +364,8 @@ public class Schema
*/
public void updateVersion()
{
- try
- {
- MessageDigest versionDigest = MessageDigest.getInstance("MD5");
-
- for (Row row : SystemKeyspace.serializedSchema())
- {
- if (invalidSchemaRow(row) || ignoredSchemaRow(row))
- continue;
-
- // we want to digest only live columns
- ColumnFamilyStore.removeDeletedColumnsOnly(row.cf, Integer.MAX_VALUE, SecondaryIndexManager.nullUpdater);
- row.cf.purgeTombstones(Integer.MAX_VALUE);
- row.cf.updateDigest(versionDigest);
- }
-
- version = UUID.nameUUIDFromBytes(versionDigest.digest());
- SystemKeyspace.updateSchemaVersion(version);
- }
- catch (Exception e)
- {
- throw new RuntimeException(e);
- }
+ version = LegacySchemaTables.calculateSchemaDigest();
+ SystemKeyspace.updateSchemaVersion(version);
}
/*
@@ -399,20 +393,202 @@ public class Schema
updateVersionAndAnnounce();
}
- public static boolean invalidSchemaRow(Row row)
+ public void addKeyspace(KSMetaData ksm)
{
- return row.cf == null || (row.cf.isMarkedForDelete() && !row.cf.hasColumns());
+ assert getKSMetaData(ksm.name) == null;
+ load(ksm);
+
+ Keyspace.open(ksm.name);
+ MigrationManager.instance.notifyCreateKeyspace(ksm);
}
- public static boolean ignoredSchemaRow(Row row)
+ public void updateKeyspace(String ksName)
{
- try
- {
- return ByteBufferUtil.string(row.key.getKey()).equals(SystemKeyspace.NAME);
- }
- catch (CharacterCodingException e)
+ KSMetaData oldKsm = getKSMetaData(ksName);
+ assert oldKsm != null;
+ KSMetaData newKsm = LegacySchemaTables.createKeyspaceFromName(ksName).cloneWith(oldKsm.cfMetaData().values(), oldKsm.userTypes);
+
+ setKeyspaceDefinition(newKsm);
+
+ Keyspace.open(ksName).createReplicationStrategy(newKsm);
+ MigrationManager.instance.notifyUpdateKeyspace(newKsm);
+ }
+
+ public void dropKeyspace(String ksName)
+ {
+ KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
+ String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
+
+ CompactionManager.instance.interruptCompactionFor(ksm.cfMetaData().values(), true);
+
+ Keyspace keyspace = Keyspace.open(ksm.name);
+
+ // remove all cfs from the keyspace instance.
+ List<UUID> droppedCfs = new ArrayList<>();
+ for (CFMetaData cfm : ksm.cfMetaData().values())
{
- throw new RuntimeException(e);
+ ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
+
+ purge(cfm);
+
+ if (DatabaseDescriptor.isAutoSnapshot())
+ cfs.snapshot(snapshotName);
+ Keyspace.open(ksm.name).dropCf(cfm.cfId);
+
+ droppedCfs.add(cfm.cfId);
}
+
+ // remove the keyspace from the static instances.
+ Keyspace.clear(ksm.name);
+ clearKeyspaceDefinition(ksm);
+
+ keyspace.writeOrder.awaitNewBarrier();
+
+ // force a new segment in the CL
+ CommitLog.instance.forceRecycleAllSegments(droppedCfs);
+
+ MigrationManager.instance.notifyDropKeyspace(ksm);
+ }
+
+ public void addTable(CFMetaData cfm)
+ {
+ assert getCFMetaData(cfm.ksName, cfm.cfName) == null;
+ KSMetaData ksm = getKSMetaData(cfm.ksName).cloneWithTableAdded(cfm);
+
+ logger.info("Loading {}", cfm);
+
+ load(cfm);
+
+ // make sure it's init-ed w/ the old definitions first,
+ // since we're going to call initCf on the new one manually
+ Keyspace.open(cfm.ksName);
+
+ setKeyspaceDefinition(ksm);
+ Keyspace.open(ksm.name).initCf(cfm.cfId, cfm.cfName, true);
+ MigrationManager.instance.notifyCreateColumnFamily(cfm);
+ }
+
+ public void updateTable(String ksName, String tableName)
+ {
+ CFMetaData cfm = getCFMetaData(ksName, tableName);
+ assert cfm != null;
+ cfm.reload();
+
+ Keyspace keyspace = Keyspace.open(cfm.ksName);
+ keyspace.getColumnFamilyStore(cfm.cfName).reload();
+ MigrationManager.instance.notifyUpdateColumnFamily(cfm);
+ }
+
+ public void dropTable(String ksName, String tableName)
+ {
+ KSMetaData ksm = getKSMetaData(ksName);
+ assert ksm != null;
+ ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(tableName);
+ assert cfs != null;
+
+ // reinitialize the keyspace.
+ CFMetaData cfm = ksm.cfMetaData().get(tableName);
+
+ purge(cfm);
+ setKeyspaceDefinition(ksm.cloneWithTableRemoved(cfm));
+
+ CompactionManager.instance.interruptCompactionFor(Arrays.asList(cfm), true);
+
+ if (DatabaseDescriptor.isAutoSnapshot())
+ cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
+ Keyspace.open(ksm.name).dropCf(cfm.cfId);
+ MigrationManager.instance.notifyDropColumnFamily(cfm);
+
+ CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
+ }
+
+ public void addType(UserType ut)
+ {
+ KSMetaData ksm = getKSMetaData(ut.keyspace);
+ assert ksm != null;
+
+ logger.info("Loading {}", ut);
+
+ ksm.userTypes.addType(ut);
+
+ MigrationManager.instance.notifyCreateUserType(ut);
+ }
+
+ public void updateType(UserType ut)
+ {
+ KSMetaData ksm = getKSMetaData(ut.keyspace);
+ assert ksm != null;
+
+ logger.info("Updating {}", ut);
+
+ ksm.userTypes.addType(ut);
+
+ MigrationManager.instance.notifyUpdateUserType(ut);
+ }
+
+ public void dropType(UserType ut)
+ {
+ KSMetaData ksm = getKSMetaData(ut.keyspace);
+ assert ksm != null;
+
+ ksm.userTypes.removeType(ut);
+
+ MigrationManager.instance.notifyDropUserType(ut);
+ }
+
+ public void addFunction(UDFunction udf)
+ {
+ logger.info("Loading {}", udf);
+
+ Functions.addFunction(udf);
+
+ MigrationManager.instance.notifyCreateFunction(udf);
+ }
+
+ public void updateFunction(UDFunction udf)
+ {
+ logger.info("Updating {}", udf);
+
+ Functions.replaceFunction(udf);
+
+ MigrationManager.instance.notifyUpdateFunction(udf);
+ }
+
+ public void dropFunction(UDFunction udf)
+ {
+ logger.info("Drop {}", udf);
+
+ // TODO: this is kind of broken as this remove all overloads of the function name
+ Functions.removeFunction(udf.name(), udf.argTypes());
+
+ MigrationManager.instance.notifyDropFunction(udf);
+ }
+
+ public void addAggregate(UDAggregate udf)
+ {
+ logger.info("Loading {}", udf);
+
+ Functions.addFunction(udf);
+
+ MigrationManager.instance.notifyCreateAggregate(udf);
+ }
+
+ public void updateAggregate(UDAggregate udf)
+ {
+ logger.info("Updating {}", udf);
+
+ Functions.replaceFunction(udf);
+
+ MigrationManager.instance.notifyUpdateAggregate(udf);
+ }
+
+ public void dropAggregate(UDAggregate udf)
+ {
+ logger.info("Drop {}", udf);
+
+ // TODO: this is kind of broken as this remove all overloads of the function name
+ Functions.removeFunction(udf.name(), udf.argTypes());
+
+ MigrationManager.instance.notifyDropAggregate(udf);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/TriggerDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/TriggerDefinition.java b/src/java/org/apache/cassandra/config/TriggerDefinition.java
index a395549..6a84379 100644
--- a/src/java/org/apache/cassandra/config/TriggerDefinition.java
+++ b/src/java/org/apache/cassandra/config/TriggerDefinition.java
@@ -18,20 +18,10 @@
*/
package org.apache.cassandra.config;
-import java.util.*;
-
import com.google.common.base.Objects;
-import org.apache.cassandra.cql3.QueryProcessor;
-import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.marshal.UTF8Type;
-
public class TriggerDefinition
{
- public static final String TRIGGER_NAME = "trigger_name";
- public static final String TRIGGER_OPTIONS = "trigger_options";
public static final String CLASS = "class";
public final String name;
@@ -51,59 +41,6 @@ public class TriggerDefinition
return new TriggerDefinition(name, classOption);
}
- /**
- * Deserialize triggers from storage-level representation.
- *
- * @param serializedTriggers storage-level partition containing the trigger definitions
- * @return the list of processed TriggerDefinitions
- */
- public static List<TriggerDefinition> fromSchema(Row serializedTriggers)
- {
- List<TriggerDefinition> triggers = new ArrayList<>();
- String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
- for (UntypedResultSet.Row row : QueryProcessor.resultify(query, serializedTriggers))
- {
- String name = row.getString(TRIGGER_NAME);
- String classOption = row.getMap(TRIGGER_OPTIONS, UTF8Type.instance, UTF8Type.instance).get(CLASS);
- triggers.add(new TriggerDefinition(name, classOption));
- }
- return triggers;
- }
-
- /**
- * Add specified trigger to the schema using given mutation.
- *
- * @param mutation The schema mutation
- * @param cfName The name of the parent ColumnFamily
- * @param timestamp The timestamp to use for the columns
- */
- public void toSchema(Mutation mutation, String cfName, long timestamp)
- {
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
-
- CFMetaData cfm = SystemKeyspace.SchemaTriggersTable;
- Composite prefix = cfm.comparator.make(cfName, name);
- CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
- adder.addMapEntry(TRIGGER_OPTIONS, CLASS, classOption);
- }
-
- /**
- * Drop specified trigger from the schema using given mutation.
- *
- * @param mutation The schema mutation
- * @param cfName The name of the parent ColumnFamily
- * @param timestamp The timestamp to use for the tombstone
- */
- public void deleteFromSchema(Mutation mutation, String cfName, long timestamp)
- {
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_TRIGGERS_TABLE);
- int ldt = (int) (System.currentTimeMillis() / 1000);
-
- Composite prefix = SystemKeyspace.SchemaTriggersTable.comparator.make(cfName, name);
- cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
- }
-
@Override
public boolean equals(Object o)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/config/UTMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/UTMetaData.java b/src/java/org/apache/cassandra/config/UTMetaData.java
index 46a7a4f..08cedee 100644
--- a/src/java/org/apache/cassandra/config/UTMetaData.java
+++ b/src/java/org/apache/cassandra/config/UTMetaData.java
@@ -20,12 +20,7 @@ package org.apache.cassandra.config;
import java.nio.ByteBuffer;
import java.util.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.utils.ByteBufferUtil;
/**
* Defined (and loaded) user types.
@@ -42,91 +37,11 @@ public final class UTMetaData
this(new HashMap<ByteBuffer, UserType>());
}
- UTMetaData(Map<ByteBuffer, UserType> types)
+ public UTMetaData(Map<ByteBuffer, UserType> types)
{
this.userTypes = types;
}
- private static UserType fromSchema(UntypedResultSet.Row row)
- {
- try
- {
- String keyspace = row.getString("keyspace_name");
- ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
- List<String> rawColumns = row.getList("field_names", UTF8Type.instance);
- List<String> rawTypes = row.getList("field_types", UTF8Type.instance);
-
- List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
- for (String rawColumn : rawColumns)
- columns.add(ByteBufferUtil.bytes(rawColumn));
-
- List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
- for (String rawType : rawTypes)
- types.add(TypeParser.parse(rawType));
-
- return new UserType(keyspace, name, columns, types);
- }
- catch (RequestValidationException e)
- {
- // If it has been written in the schema, it should be valid
- throw new AssertionError();
- }
- }
-
- public static Map<ByteBuffer, UserType> fromSchema(Row row)
- {
- UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_USER_TYPES_TABLE, row);
- Map<ByteBuffer, UserType> types = new HashMap<>(results.size());
- for (UntypedResultSet.Row result : results)
- {
- UserType type = fromSchema(result);
- types.put(type.name, type);
- }
- return types;
- }
-
- public static Mutation toSchema(UserType newType, long timestamp)
- {
- return toSchema(new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(newType.keyspace)), newType, timestamp);
- }
-
- public static Mutation toSchema(Mutation mutation, UserType newType, long timestamp)
- {
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
-
- Composite prefix = SystemKeyspace.SchemaUserTypesTable.comparator.make(newType.name);
- CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
- adder.resetCollection("field_names");
- adder.resetCollection("field_types");
-
- for (int i = 0; i < newType.size(); i++)
- {
- adder.addListEntry("field_names", newType.fieldName(i));
- adder.addListEntry("field_types", newType.fieldType(i).toString());
- }
- return mutation;
- }
-
- public Mutation toSchema(Mutation mutation, long timestamp)
- {
- for (UserType ut : userTypes.values())
- toSchema(mutation, ut, timestamp);
- return mutation;
- }
-
- public static Mutation dropFromSchema(UserType droppedType, long timestamp)
- {
- Mutation mutation = new Mutation(SystemKeyspace.NAME, SystemKeyspace.getSchemaKSKey(droppedType.keyspace));
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_USER_TYPES_TABLE);
- int ldt = (int) (System.currentTimeMillis() / 1000);
-
- Composite prefix = SystemKeyspace.SchemaUserTypesTable.comparator.make(droppedType.name);
- cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
- return mutation;
- }
-
public UserType getType(ByteBuffer typeName)
{
return userTypes.get(typeName);
@@ -134,11 +49,11 @@ public final class UTMetaData
public Map<ByteBuffer, UserType> getAllTypes()
{
- // Copy to avoid concurrent modification while iterating. Not intended to be called on a criticial path anyway
+ // Copy to avoid concurrent modification while iterating. Not intended to be called on a critical path anyway
return new HashMap<>(userTypes);
}
- // This is *not* thread safe but is only called in DefsTables that is synchronized.
+ // This is *not* thread safe but is only called in Schema that is synchronized.
public void addType(UserType type)
{
UserType old = userTypes.get(type.name);
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/Functions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/Functions.java b/src/java/org/apache/cassandra/cql3/functions/Functions.java
index 7d94e47..b55ebc5 100644
--- a/src/java/org/apache/cassandra/cql3/functions/Functions.java
+++ b/src/java/org/apache/cassandra/cql3/functions/Functions.java
@@ -22,12 +22,9 @@ import java.util.Collection;
import java.util.List;
import com.google.common.collect.ArrayListMultimap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.service.IMigrationListener;
@@ -35,16 +32,11 @@ import org.apache.cassandra.service.MigrationManager;
public abstract class Functions
{
- private static final Logger logger = LoggerFactory.getLogger(Functions.class);
-
// We special case the token function because that's the only function whose argument types actually
// depend on the table on which the function is called. Because it's the sole exception, it's easier
// to handle it as a special case.
private static final FunctionName TOKEN_FUNCTION_NAME = FunctionName.nativeFunction("token");
- private static final String SELECT_UD_FUNCTION = "SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.SCHEMA_FUNCTIONS_TABLE;
- private static final String SELECT_UD_AGGREGATE = "SELECT * FROM " + SystemKeyspace.NAME + '.' + SystemKeyspace.SCHEMA_AGGREGATES_TABLE;
-
private Functions() {}
private static final ArrayListMultimap<FunctionName, Function> declared = ArrayListMultimap.create();
@@ -96,18 +88,6 @@ public abstract class Functions
declared.put(fun.name(), fun);
}
- /**
- * Loading existing UDFs from the schema.
- */
- public static void loadUDFFromSchema()
- {
- logger.debug("Loading UDFs");
- for (UntypedResultSet.Row row : QueryProcessor.executeOnceInternal(SELECT_UD_FUNCTION))
- addFunction(UDFunction.fromSchema(row));
- for (UntypedResultSet.Row row : QueryProcessor.executeOnceInternal(SELECT_UD_AGGREGATE))
- addFunction(UDAggregate.fromSchema(row));
- }
-
public static ColumnSpecification makeArgSpec(String receiverKs, String receiverCf, Function fun, int i)
{
return new ColumnSpecification(receiverKs,
@@ -270,7 +250,7 @@ public abstract class Functions
return sb.toString();
}
- // This is *not* thread safe but is only called in DefsTables that is synchronized.
+ // This is *not* thread safe but is only called in SchemaTables that is synchronized.
public static void addFunction(AbstractFunction fun)
{
// We shouldn't get there unless that function don't exist
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
index 5b1f5bd..e4e6a55 100644
--- a/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
+++ b/src/java/org/apache/cassandra/cql3/functions/JavaSourceUDFFactory.java
@@ -76,9 +76,8 @@ public final class JavaSourceUDFFactory
// It is separated to allow return type and argument type checks during compile time via javassist.
String codeExecInt = generateExecuteInternalMethod(argNames, body, javaReturnType, javaParamTypes);
- if (logger.isDebugEnabled())
- logger.debug("Generating java source UDF for {} with following c'tor and functions:\n{}\n{}\n{}",
- name, codeCtor, codeExecInt, codeExec);
+ logger.debug("Generating java source UDF for {} with following c'tor and functions:\n{}\n{}\n{}",
+ name, codeCtor, codeExecInt, codeExec);
try
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
index f259265..e9c33ba 100644
--- a/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
+++ b/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java
@@ -24,12 +24,7 @@ import com.google.common.base.Objects;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.Composite;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.TypeParser;
-import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.exceptions.*;
/**
@@ -58,6 +53,45 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
this.initcond = initcond;
}
+ public static UDAggregate create(FunctionName name,
+ List<AbstractType<?>> argTypes,
+ AbstractType<?> returnType,
+ FunctionName stateFunc,
+ FunctionName finalFunc,
+ AbstractType<?> stateType,
+ ByteBuffer initcond)
+ throws InvalidRequestException
+ {
+ List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
+ stateTypes.add(stateType);
+ stateTypes.addAll(argTypes);
+ List<AbstractType<?>> finalTypes = Collections.<AbstractType<?>>singletonList(stateType);
+ return new UDAggregate(name,
+ argTypes,
+ returnType,
+ resolveScalar(name, stateFunc, stateTypes),
+ finalFunc != null ? resolveScalar(name, finalFunc, finalTypes) : null,
+ initcond);
+ }
+
+ public static UDAggregate createBroken(FunctionName name,
+ List<AbstractType<?>> argTypes,
+ AbstractType<?> returnType,
+ ByteBuffer initcond,
+ final InvalidRequestException reason)
+ {
+ return new UDAggregate(name, argTypes, returnType, null, null, initcond)
+ {
+ public Aggregate newAggregate() throws InvalidRequestException
+ {
+ throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. "
+ + "Please see the server log for more details",
+ this,
+ reason.getMessage()));
+ }
+ };
+ }
+
public boolean hasReferenceTo(Function function)
{
return stateFunction == function || finalFunction == function;
@@ -85,6 +119,26 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
return false;
}
+ public ScalarFunction stateFunction()
+ {
+ return stateFunction;
+ }
+
+ public ScalarFunction finalFunction()
+ {
+ return finalFunction;
+ }
+
+ public ByteBuffer initialCondition()
+ {
+ return initcond;
+ }
+
+ public AbstractType<?> stateType()
+ {
+ return stateType;
+ }
+
public Aggregate newAggregate() throws InvalidRequestException
{
return new Aggregate()
@@ -128,134 +182,6 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
return (ScalarFunction) func;
}
- private static Mutation makeSchemaMutation(FunctionName name)
- {
- UTF8Type kv = (UTF8Type)SystemKeyspace.SchemaAggregatesTable.getKeyValidator();
- return new Mutation(SystemKeyspace.NAME, kv.decompose(name.keyspace));
- }
-
- public Mutation toSchemaDrop(long timestamp)
- {
- Mutation mutation = makeSchemaMutation(name);
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_AGGREGATES_TABLE);
-
- Composite prefix = SystemKeyspace.SchemaAggregatesTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
- int ldt = (int) (System.currentTimeMillis() / 1000);
- cf.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
-
- return mutation;
- }
-
- public static Map<Composite, UDAggregate> fromSchema(Row row)
- {
- UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system." + SystemKeyspace.SCHEMA_AGGREGATES_TABLE, row);
- Map<Composite, UDAggregate> udfs = new HashMap<>(results.size());
- for (UntypedResultSet.Row result : results)
- udfs.put(SystemKeyspace.SchemaAggregatesTable.comparator.make(result.getString("aggregate_name"), result.getBlob("signature")),
- fromSchema(result));
- return udfs;
- }
-
- public Mutation toSchemaUpdate(long timestamp)
- {
- Mutation mutation = makeSchemaMutation(name);
- ColumnFamily cf = mutation.addOrGet(SystemKeyspace.SCHEMA_AGGREGATES_TABLE);
-
- Composite prefix = SystemKeyspace.SchemaAggregatesTable.comparator.make(name.name, UDHelper.computeSignature(argTypes));
- CFRowAdder adder = new CFRowAdder(cf, prefix, timestamp);
-
- adder.resetCollection("argument_types");
- adder.add("return_type", returnType.toString());
- adder.add("state_func", stateFunction.name().name);
- if (stateType != null)
- adder.add("state_type", stateType.toString());
- if (finalFunction != null)
- adder.add("final_func", finalFunction.name().name);
- if (initcond != null)
- adder.add("initcond", initcond);
-
- for (AbstractType<?> argType : argTypes)
- adder.addListEntry("argument_types", argType.toString());
-
- return mutation;
- }
-
- public static UDAggregate fromSchema(UntypedResultSet.Row row)
- {
- String ksName = row.getString("keyspace_name");
- String functionName = row.getString("aggregate_name");
- FunctionName name = new FunctionName(ksName, functionName);
-
- List<String> types = row.getList("argument_types", UTF8Type.instance);
-
- List<AbstractType<?>> argTypes;
- if (types == null)
- {
- argTypes = Collections.emptyList();
- }
- else
- {
- argTypes = new ArrayList<>(types.size());
- for (String type : types)
- argTypes.add(parseType(type));
- }
-
- AbstractType<?> returnType = parseType(row.getString("return_type"));
-
- FunctionName stateFunc = new FunctionName(ksName, row.getString("state_func"));
- FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
- AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
- ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
-
- try
- {
- return create(name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
- }
- catch (InvalidRequestException reason)
- {
- return createBroken(name, argTypes, returnType, initcond, reason);
- }
- }
-
- private static UDAggregate createBroken(FunctionName name, List<AbstractType<?>> argTypes, AbstractType<?> returnType,
- ByteBuffer initcond, final InvalidRequestException reason)
- {
- return new UDAggregate(name, argTypes, returnType, null, null, initcond) {
- public Aggregate newAggregate() throws InvalidRequestException
- {
- throw new InvalidRequestException(String.format("Aggregate '%s' exists but hasn't been loaded successfully for the following reason: %s. "
- + "Please see the server log for more details", this, reason.getMessage()));
- }
- };
- }
-
- private static UDAggregate create(FunctionName name, List<AbstractType<?>> argTypes, AbstractType<?> returnType,
- FunctionName stateFunc, FunctionName finalFunc, AbstractType<?> stateType, ByteBuffer initcond)
- throws InvalidRequestException
- {
- List<AbstractType<?>> stateTypes = new ArrayList<>(argTypes.size() + 1);
- stateTypes.add(stateType);
- stateTypes.addAll(argTypes);
- List<AbstractType<?>> finalTypes = Collections.<AbstractType<?>>singletonList(stateType);
- return new UDAggregate(name, argTypes, returnType,
- resolveScalar(name, stateFunc, stateTypes),
- finalFunc != null ? resolveScalar(name, finalFunc, finalTypes) : null,
- initcond);
- }
-
- private static AbstractType<?> parseType(String str)
- {
- // We only use this when reading the schema where we shouldn't get an error
- try
- {
- return TypeParser.parse(str);
- }
- catch (SyntaxException | ConfigurationException e)
- {
- throw new RuntimeException(e);
- }
- }
-
@Override
public boolean equals(Object o)
{
@@ -263,13 +189,13 @@ public class UDAggregate extends AbstractFunction implements AggregateFunction
return false;
UDAggregate that = (UDAggregate) o;
- return Objects.equal(this.name, that.name)
- && Functions.typeEquals(this.argTypes, that.argTypes)
- && Functions.typeEquals(this.returnType, that.returnType)
- && Objects.equal(this.stateFunction, that.stateFunction)
- && Objects.equal(this.finalFunction, that.finalFunction)
- && Objects.equal(this.stateType, that.stateType)
- && Objects.equal(this.initcond, that.initcond);
+ return Objects.equal(name, that.name)
+ && Functions.typeEquals(argTypes, that.argTypes)
+ && Functions.typeEquals(returnType, that.returnType)
+ && Objects.equal(stateFunction, that.stateFunction)
+ && Objects.equal(finalFunction, that.finalFunction)
+ && Objects.equal(stateType, that.stateType)
+ && Objects.equal(initcond, that.initcond);
}
@Override
[2/5] cassandra git commit: Isolate schema serializaton code
Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/LegacySchemaTables.java b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
new file mode 100644
index 0000000..047698c
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/LegacySchemaTables.java
@@ -0,0 +1,1480 @@
+/*
+ * 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.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.*;
+import org.apache.cassandra.cql3.functions.*;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.composites.Composite;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.SyntaxException;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
+import static org.apache.cassandra.utils.FBUtilities.fromJsonMap;
+import static org.apache.cassandra.utils.FBUtilities.json;
+
+/** system.schema_* tables used to store keyspace/table/type attributes prior to C* 3.0 */
+public class LegacySchemaTables
+{
+ private static final Logger logger = LoggerFactory.getLogger(LegacySchemaTables.class);
+
+ public static final String KEYSPACES = "schema_keyspaces";
+ public static final String COLUMNFAMILIES = "schema_columnfamilies";
+ public static final String COLUMNS = "schema_columns";
+ public static final String TRIGGERS = "schema_triggers";
+ public static final String USERTYPES = "schema_usertypes";
+ public static final String FUNCTIONS = "schema_functions";
+ public static final String AGGREGATES = "schema_aggregates";
+
+ public static final List<String> ALL = Arrays.asList(KEYSPACES, COLUMNFAMILIES, COLUMNS, TRIGGERS, USERTYPES, FUNCTIONS, AGGREGATES);
+
+ private static final CFMetaData Keyspaces =
+ compile(KEYSPACES,
+ "keyspace definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "durable_writes boolean,"
+ + "strategy_class text,"
+ + "strategy_options text,"
+ + "PRIMARY KEY ((keyspace_name))) "
+ + "WITH COMPACT STORAGE");
+
+ private static final CFMetaData Columnfamilies =
+ compile(COLUMNFAMILIES,
+ "table definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "columnfamily_name text,"
+ + "bloom_filter_fp_chance double,"
+ + "caching text,"
+ + "cf_id uuid," // post-2.1 UUID cfid
+ + "comment text,"
+ + "compaction_strategy_class text,"
+ + "compaction_strategy_options text,"
+ + "comparator text,"
+ + "compression_parameters text,"
+ + "default_time_to_live int,"
+ + "default_validator text,"
+ + "dropped_columns map<text, bigint>,"
+ + "gc_grace_seconds int,"
+ + "is_dense boolean,"
+ + "key_validator text,"
+ + "local_read_repair_chance double,"
+ + "max_compaction_threshold int,"
+ + "max_index_interval int,"
+ + "memtable_flush_period_in_ms int,"
+ + "min_compaction_threshold int,"
+ + "min_index_interval int,"
+ + "read_repair_chance double,"
+ + "speculative_retry text,"
+ + "subcomparator text,"
+ + "type text,"
+ + "PRIMARY KEY ((keyspace_name), columnfamily_name))");
+
+ private static final CFMetaData Columns =
+ compile(COLUMNS,
+ "column definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "columnfamily_name text,"
+ + "column_name text,"
+ + "component_index int,"
+ + "index_name text,"
+ + "index_options text,"
+ + "index_type text,"
+ + "type text,"
+ + "validator text,"
+ + "PRIMARY KEY ((keyspace_name), columnfamily_name, column_name))");
+
+ private static final CFMetaData Triggers =
+ compile(TRIGGERS,
+ "trigger definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "columnfamily_name text,"
+ + "trigger_name text,"
+ + "trigger_options map<text, text>,"
+ + "PRIMARY KEY ((keyspace_name), columnfamily_name, trigger_name))");
+
+ private static final CFMetaData Usertypes =
+ compile(USERTYPES,
+ "user defined type definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "type_name text,"
+ + "field_names list<text>,"
+ + "field_types list<text>,"
+ + "PRIMARY KEY ((keyspace_name), type_name))");
+
+ private static final CFMetaData Functions =
+ compile(FUNCTIONS,
+ "user defined function definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "function_name text,"
+ + "signature blob,"
+ + "argument_names list<text>,"
+ + "argument_types list<text>,"
+ + "body text,"
+ + "is_deterministic boolean,"
+ + "language text,"
+ + "return_type text,"
+ + "PRIMARY KEY ((keyspace_name), function_name, signature))");
+
+ private static final CFMetaData Aggregates =
+ compile(AGGREGATES,
+ "user defined aggregate definitions",
+ "CREATE TABLE %s ("
+ + "keyspace_name text,"
+ + "aggregate_name text,"
+ + "signature blob,"
+ + "argument_types list<text>,"
+ + "final_func text,"
+ + "initcond blob,"
+ + "return_type text,"
+ + "state_func text,"
+ + "state_type text,"
+ + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))");
+
+ public static final List<CFMetaData> All = Arrays.asList(Keyspaces, Columnfamilies, Columns, Triggers, Usertypes, Functions, Aggregates);
+
+ private static CFMetaData compile(String name, String description, String schema)
+ {
+ return CFMetaData.compile(String.format(schema, name), SystemKeyspace.NAME)
+ .comment(description)
+ .gcGraceSeconds((int) TimeUnit.DAYS.toSeconds(7));
+ }
+
+ /** add entries to system.schema_* for the hardcoded system definitions */
+ public static void saveSystemKeyspaceSchema()
+ {
+ KSMetaData keyspace = Schema.instance.getKSMetaData(SystemKeyspace.NAME);
+ // delete old, possibly obsolete entries in schema tables
+ for (String table : ALL)
+ executeOnceInternal(String.format("DELETE FROM system.%s WHERE keyspace_name = ?", table), keyspace.name);
+ // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
+ makeCreateKeyspaceMutation(keyspace, FBUtilities.timestampMicros() + 1).apply();
+ }
+
+ public static Collection<KSMetaData> readSchemaFromSystemTables()
+ {
+ List<Row> serializedSchema = getSchemaPartitionsForTable(KEYSPACES);
+
+ List<KSMetaData> keyspaces = new ArrayList<>(serializedSchema.size());
+
+ for (Row partition : serializedSchema)
+ {
+ if (isEmptySchemaPartition(partition) || isSystemKeyspaceSchemaPartition(partition))
+ continue;
+
+ keyspaces.add(createKeyspaceFromSchemaPartitions(partition,
+ readSchemaPartitionForKeyspace(COLUMNFAMILIES, partition.key),
+ readSchemaPartitionForKeyspace(USERTYPES, partition.key)));
+
+ // Will be moved away in #6717
+ for (UDFunction function : createFunctionsFromFunctionsPartition(readSchemaPartitionForKeyspace(FUNCTIONS, partition.key)).values())
+ org.apache.cassandra.cql3.functions.Functions.addFunction(function);
+
+ // Will be moved away in #6717
+ for (UDAggregate aggregate : createAggregatesFromAggregatesPartition(readSchemaPartitionForKeyspace(AGGREGATES, partition.key)).values())
+ org.apache.cassandra.cql3.functions.Functions.addFunction(aggregate);
+ }
+
+ return keyspaces;
+ }
+
+ public static void truncateSchemaTables()
+ {
+ for (String table : ALL)
+ getSchemaCFS(table).truncateBlocking();
+ }
+
+ private static void flushSchemaTables()
+ {
+ for (String table : ALL)
+ SystemKeyspace.forceBlockingFlush(table);
+ }
+
+ /**
+ * Read schema from system keyspace and calculate MD5 digest of every row, resulting digest
+ * will be converted into UUID which would act as content-based version of the schema.
+ */
+ public static UUID calculateSchemaDigest()
+ {
+ MessageDigest digest;
+ try
+ {
+ digest = MessageDigest.getInstance("MD5");
+ }
+ catch (NoSuchAlgorithmException e)
+ {
+ throw new RuntimeException(e);
+ }
+
+ for (String table : ALL)
+ {
+ for (Row partition : getSchemaPartitionsForTable(table))
+ {
+ if (isEmptySchemaPartition(partition) || isSystemKeyspaceSchemaPartition(partition))
+ continue;
+
+ // we want to digest only live columns
+ ColumnFamilyStore.removeDeletedColumnsOnly(partition.cf, Integer.MAX_VALUE, SecondaryIndexManager.nullUpdater);
+ partition.cf.purgeTombstones(Integer.MAX_VALUE);
+ partition.cf.updateDigest(digest);
+ }
+ }
+
+ return UUID.nameUUIDFromBytes(digest.digest());
+ }
+
+ /**
+ * @param schemaTableName The name of the table responsible for part of the schema
+ * @return CFS responsible to hold low-level serialized schema
+ */
+ private static ColumnFamilyStore getSchemaCFS(String schemaTableName)
+ {
+ return Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(schemaTableName);
+ }
+
+ /**
+ * @param schemaTableName The name of the table responsible for part of the schema.
+ * @return low-level schema representation
+ */
+ private static List<Row> getSchemaPartitionsForTable(String schemaTableName)
+ {
+ Token minToken = StorageService.getPartitioner().getMinimumToken();
+ return getSchemaCFS(schemaTableName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
+ null,
+ new IdentityQueryFilter(),
+ Integer.MAX_VALUE,
+ System.currentTimeMillis());
+ }
+
+ public static Collection<Mutation> convertSchemaToMutations()
+ {
+ Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
+
+ for (String table : ALL)
+ convertSchemaToMutations(mutationMap, table);
+
+ return mutationMap.values();
+ }
+
+ private static void convertSchemaToMutations(Map<DecoratedKey, Mutation> mutationMap, String schemaTableName)
+ {
+ for (Row partition : getSchemaPartitionsForTable(schemaTableName))
+ {
+ if (isSystemKeyspaceSchemaPartition(partition))
+ continue;
+
+ Mutation mutation = mutationMap.get(partition.key);
+ if (mutation == null)
+ {
+ mutation = new Mutation(SystemKeyspace.NAME, partition.key.getKey());
+ mutationMap.put(partition.key, mutation);
+ }
+
+ mutation.add(partition.cf);
+ }
+ }
+
+ private static Map<DecoratedKey, ColumnFamily> readSchemaForKeyspaces(String schemaTableName, Set<String> keyspaceNames)
+ {
+ Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
+
+ for (String keyspaceName : keyspaceNames)
+ {
+ Row schemaEntity = readSchemaPartitionForKeyspace(schemaTableName, keyspaceName);
+ if (schemaEntity.cf != null)
+ schema.put(schemaEntity.key, schemaEntity.cf);
+ }
+
+ return schema;
+ }
+
+ private static ByteBuffer getSchemaKSKey(String ksName)
+ {
+ return AsciiType.instance.fromString(ksName);
+ }
+
+ private static Row readSchemaPartitionForKeyspace(String schemaTableName, String keyspaceName)
+ {
+ DecoratedKey keyspaceKey = StorageService.getPartitioner().decorateKey(getSchemaKSKey(keyspaceName));
+ return readSchemaPartitionForKeyspace(schemaTableName, keyspaceKey);
+ }
+
+ private static Row readSchemaPartitionForKeyspace(String schemaTableName, DecoratedKey keyspaceKey)
+ {
+ QueryFilter filter = QueryFilter.getIdentityFilter(keyspaceKey, schemaTableName, System.currentTimeMillis());
+ return new Row(keyspaceKey, getSchemaCFS(schemaTableName).getColumnFamily(filter));
+ }
+
+ private static Row readSchemaPartitionForTable(String schemaTableName, String keyspaceName, String tableName)
+ {
+ DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(keyspaceName));
+ ColumnFamilyStore store = getSchemaCFS(schemaTableName);
+ Composite prefix = store.getComparator().make(tableName);
+ ColumnFamily cells = store.getColumnFamily(key, prefix, prefix.end(), false, Integer.MAX_VALUE, System.currentTimeMillis());
+ return new Row(key, cells);
+ }
+
+ private static boolean isEmptySchemaPartition(Row partition)
+ {
+ return partition.cf == null || (partition.cf.isMarkedForDelete() && !partition.cf.hasColumns());
+ }
+
+ private static boolean isSystemKeyspaceSchemaPartition(Row partition)
+ {
+ return getSchemaKSKey(SystemKeyspace.NAME).equals(partition.key.getKey());
+ }
+
+ /**
+ * Merge remote schema in form of mutations with local and mutate ks/cf metadata objects
+ * (which also involves fs operations on add/drop ks/cf)
+ *
+ * @param mutations the schema changes to apply
+ *
+ * @throws ConfigurationException If one of metadata attributes has invalid value
+ * @throws IOException If data was corrupted during transportation or failed to apply fs operations
+ */
+ public static synchronized void mergeSchema(Collection<Mutation> mutations) throws ConfigurationException, IOException
+ {
+ mergeSchema(mutations, true);
+ Schema.instance.updateVersionAndAnnounce();
+ }
+
+ public static synchronized void mergeSchema(Collection<Mutation> mutations, boolean doFlush) throws IOException
+ {
+ // compare before/after schemas of the affected keyspaces only
+ Set<String> keyspaces = new HashSet<>(mutations.size());
+ for (Mutation mutation : mutations)
+ keyspaces.add(ByteBufferUtil.string(mutation.key()));
+
+ // current state of the schema
+ Map<DecoratedKey, ColumnFamily> oldKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
+ Map<DecoratedKey, ColumnFamily> oldColumnFamilies = readSchemaForKeyspaces(COLUMNFAMILIES, keyspaces);
+ Map<DecoratedKey, ColumnFamily> oldTypes = readSchemaForKeyspaces(USERTYPES, keyspaces);
+ Map<DecoratedKey, ColumnFamily> oldFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
+ Map<DecoratedKey, ColumnFamily> oldAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
+
+ for (Mutation mutation : mutations)
+ mutation.apply();
+
+ if (doFlush)
+ flushSchemaTables();
+
+ // with new data applied
+ Map<DecoratedKey, ColumnFamily> newKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
+ Map<DecoratedKey, ColumnFamily> newColumnFamilies = readSchemaForKeyspaces(COLUMNFAMILIES, keyspaces);
+ Map<DecoratedKey, ColumnFamily> newTypes = readSchemaForKeyspaces(USERTYPES, keyspaces);
+ Map<DecoratedKey, ColumnFamily> newFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
+ Map<DecoratedKey, ColumnFamily> newAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
+
+ Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
+ mergeTables(oldColumnFamilies, newColumnFamilies);
+ mergeTypes(oldTypes, newTypes);
+ mergeFunctions(oldFunctions, newFunctions);
+ mergeAggregates(oldAggregates, newAggregates);
+
+ // it is safe to drop a keyspace only when all nested ColumnFamilies where deleted
+ for (String keyspaceToDrop : keyspacesToDrop)
+ Schema.instance.dropKeyspace(keyspaceToDrop);
+ }
+
+ private static Set<String> mergeKeyspaces(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+ {
+ List<Row> created = new ArrayList<>();
+ List<String> altered = new ArrayList<>();
+ Set<String> dropped = new HashSet<>();
+
+ /*
+ * - we don't care about entriesOnlyOnLeft() or entriesInCommon(), because only the changes are of interest to us
+ * - of all entriesOnlyOnRight(), we only care about ones that have live columns; it's possible to have a ColumnFamily
+ * there that only has the top-level deletion, if:
+ * a) a pushed DROP KEYSPACE change for a keyspace hadn't ever made it to this node in the first place
+ * b) a pulled dropped keyspace that got dropped before it could find a way to this node
+ * - of entriesDiffering(), we don't care about the scenario where both pre and post-values have zero live columns:
+ * that means that a keyspace had been recreated and dropped, and the recreated keyspace had never found a way
+ * to this node
+ */
+ MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+ if (entry.getValue().hasColumns())
+ created.add(new Row(entry.getKey(), entry.getValue()));
+
+ for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+ {
+ String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+ ColumnFamily pre = entry.getValue().leftValue();
+ ColumnFamily post = entry.getValue().rightValue();
+
+ if (pre.hasColumns() && post.hasColumns())
+ altered.add(keyspaceName);
+ else if (pre.hasColumns())
+ dropped.add(keyspaceName);
+ else if (post.hasColumns()) // a (re)created keyspace
+ created.add(new Row(entry.getKey(), post));
+ }
+
+ for (Row row : created)
+ Schema.instance.addKeyspace(createKeyspaceFromSchemaPartition(row));
+ for (String name : altered)
+ Schema.instance.updateKeyspace(name);
+ return dropped;
+ }
+
+ // see the comments for mergeKeyspaces()
+ private static void mergeTables(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+ {
+ List<CFMetaData> created = new ArrayList<>();
+ List<CFMetaData> altered = new ArrayList<>();
+ List<CFMetaData> dropped = new ArrayList<>();
+
+ MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+ if (entry.getValue().hasColumns())
+ created.addAll(createTablesFromTablesPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+ for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+ {
+ String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+ ColumnFamily pre = entry.getValue().leftValue();
+ ColumnFamily post = entry.getValue().rightValue();
+
+ if (pre.hasColumns() && post.hasColumns())
+ {
+ MapDifference<String, CFMetaData> delta =
+ Maps.difference(Schema.instance.getKSMetaData(keyspaceName).cfMetaData(),
+ createTablesFromTablesPartition(new Row(entry.getKey(), post)));
+
+ dropped.addAll(delta.entriesOnlyOnLeft().values());
+ created.addAll(delta.entriesOnlyOnRight().values());
+ Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<CFMetaData>, CFMetaData>()
+ {
+ public CFMetaData apply(MapDifference.ValueDifference<CFMetaData> pair)
+ {
+ return pair.rightValue();
+ }
+ }));
+ }
+ else if (pre.hasColumns())
+ {
+ dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).cfMetaData().values());
+ }
+ else if (post.hasColumns())
+ {
+ created.addAll(createTablesFromTablesPartition(new Row(entry.getKey(), post)).values());
+ }
+ }
+
+ for (CFMetaData cfm : created)
+ Schema.instance.addTable(cfm);
+ for (CFMetaData cfm : altered)
+ Schema.instance.updateTable(cfm.ksName, cfm.cfName);
+ for (CFMetaData cfm : dropped)
+ Schema.instance.dropTable(cfm.ksName, cfm.cfName);
+ }
+
+ // see the comments for mergeKeyspaces()
+ private static void mergeTypes(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+ {
+ List<UserType> created = new ArrayList<>();
+ List<UserType> altered = new ArrayList<>();
+ List<UserType> dropped = new ArrayList<>();
+
+ MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+ // New keyspace with types
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+ if (entry.getValue().hasColumns())
+ created.addAll(createTypesFromPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+ for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+ {
+ String keyspaceName = AsciiType.instance.compose(entry.getKey().getKey());
+
+ ColumnFamily pre = entry.getValue().leftValue();
+ ColumnFamily post = entry.getValue().rightValue();
+
+ if (pre.hasColumns() && post.hasColumns())
+ {
+ MapDifference<ByteBuffer, UserType> delta =
+ Maps.difference(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes(),
+ createTypesFromPartition(new Row(entry.getKey(), post)));
+
+ dropped.addAll(delta.entriesOnlyOnLeft().values());
+ created.addAll(delta.entriesOnlyOnRight().values());
+ Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UserType>, UserType>()
+ {
+ public UserType apply(MapDifference.ValueDifference<UserType> pair)
+ {
+ return pair.rightValue();
+ }
+ }));
+ }
+ else if (pre.hasColumns())
+ {
+ dropped.addAll(Schema.instance.getKSMetaData(keyspaceName).userTypes.getAllTypes().values());
+ }
+ else if (post.hasColumns())
+ {
+ created.addAll(createTypesFromPartition(new Row(entry.getKey(), post)).values());
+ }
+ }
+
+ for (UserType type : created)
+ Schema.instance.addType(type);
+ for (UserType type : altered)
+ Schema.instance.updateType(type);
+ for (UserType type : dropped)
+ Schema.instance.dropType(type);
+ }
+
+ // see the comments for mergeKeyspaces()
+ private static void mergeFunctions(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+ {
+ List<UDFunction> created = new ArrayList<>();
+ List<UDFunction> altered = new ArrayList<>();
+ List<UDFunction> dropped = new ArrayList<>();
+
+ MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+ // New keyspace with functions
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+ if (entry.getValue().hasColumns())
+ created.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+ for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+ {
+ ColumnFamily pre = entry.getValue().leftValue();
+ ColumnFamily post = entry.getValue().rightValue();
+
+ if (pre.hasColumns() && post.hasColumns())
+ {
+ MapDifference<ByteBuffer, UDFunction> delta =
+ Maps.difference(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), pre)),
+ createFunctionsFromFunctionsPartition(new Row(entry.getKey(), post)));
+
+ dropped.addAll(delta.entriesOnlyOnLeft().values());
+ created.addAll(delta.entriesOnlyOnRight().values());
+ Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDFunction>, UDFunction>()
+ {
+ public UDFunction apply(MapDifference.ValueDifference<UDFunction> pair)
+ {
+ return pair.rightValue();
+ }
+ }));
+ }
+ else if (pre.hasColumns())
+ {
+ dropped.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), pre)).values());
+ }
+ else if (post.hasColumns())
+ {
+ created.addAll(createFunctionsFromFunctionsPartition(new Row(entry.getKey(), post)).values());
+ }
+ }
+
+ for (UDFunction udf : created)
+ Schema.instance.addFunction(udf);
+ for (UDFunction udf : altered)
+ Schema.instance.updateFunction(udf);
+ for (UDFunction udf : dropped)
+ Schema.instance.dropFunction(udf);
+ }
+
+ // see the comments for mergeKeyspaces()
+ private static void mergeAggregates(Map<DecoratedKey, ColumnFamily> before, Map<DecoratedKey, ColumnFamily> after)
+ {
+ List<UDAggregate> created = new ArrayList<>();
+ List<UDAggregate> altered = new ArrayList<>();
+ List<UDAggregate> dropped = new ArrayList<>();
+
+ MapDifference<DecoratedKey, ColumnFamily> diff = Maps.difference(before, after);
+
+ // New keyspace with functions
+ for (Map.Entry<DecoratedKey, ColumnFamily> entry : diff.entriesOnlyOnRight().entrySet())
+ if (entry.getValue().hasColumns())
+ created.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), entry.getValue())).values());
+
+ for (Map.Entry<DecoratedKey, MapDifference.ValueDifference<ColumnFamily>> entry : diff.entriesDiffering().entrySet())
+ {
+ ColumnFamily pre = entry.getValue().leftValue();
+ ColumnFamily post = entry.getValue().rightValue();
+
+ if (pre.hasColumns() && post.hasColumns())
+ {
+ MapDifference<ByteBuffer, UDAggregate> delta =
+ Maps.difference(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), pre)),
+ createAggregatesFromAggregatesPartition(new Row(entry.getKey(), post)));
+
+ dropped.addAll(delta.entriesOnlyOnLeft().values());
+ created.addAll(delta.entriesOnlyOnRight().values());
+ Iterables.addAll(altered, Iterables.transform(delta.entriesDiffering().values(), new Function<MapDifference.ValueDifference<UDAggregate>, UDAggregate>()
+ {
+ public UDAggregate apply(MapDifference.ValueDifference<UDAggregate> pair)
+ {
+ return pair.rightValue();
+ }
+ }));
+ }
+ else if (pre.hasColumns())
+ {
+ dropped.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), pre)).values());
+ }
+ else if (post.hasColumns())
+ {
+ created.addAll(createAggregatesFromAggregatesPartition(new Row(entry.getKey(), post)).values());
+ }
+ }
+
+ for (UDAggregate udf : created)
+ Schema.instance.addAggregate(udf);
+ for (UDAggregate udf : altered)
+ Schema.instance.updateAggregate(udf);
+ for (UDAggregate udf : dropped)
+ Schema.instance.dropAggregate(udf);
+ }
+
+ /*
+ * Keyspace metadata serialization/deserialization.
+ */
+
+ public static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, long timestamp)
+ {
+ return makeCreateKeyspaceMutation(keyspace, timestamp, true);
+ }
+
+ private static Mutation makeCreateKeyspaceMutation(KSMetaData keyspace, long timestamp, boolean withTablesAndTypesAndFunctions)
+ {
+ Mutation mutation = new Mutation(SystemKeyspace.NAME, getSchemaKSKey(keyspace.name));
+ ColumnFamily cells = mutation.addOrGet(Keyspaces);
+ CFRowAdder adder = new CFRowAdder(cells, Keyspaces.comparator.builder().build(), timestamp);
+
+ adder.add("durable_writes", keyspace.durableWrites);
+ adder.add("strategy_class", keyspace.strategyClass.getName());
+ adder.add("strategy_options", json(keyspace.strategyOptions));
+
+ if (withTablesAndTypesAndFunctions)
+ {
+ for (UserType type : keyspace.userTypes.getAllTypes().values())
+ addTypeToSchemaMutation(type, timestamp, mutation);
+
+ for (CFMetaData table : keyspace.cfMetaData().values())
+ addTableToSchemaMutation(table, timestamp, true, mutation);
+ }
+
+ return mutation;
+ }
+
+ public static Mutation makeDropKeyspaceMutation(KSMetaData keyspace, long timestamp)
+ {
+ Mutation mutation = new Mutation(SystemKeyspace.NAME, getSchemaKSKey(keyspace.name));
+ for (String schemaTable : ALL)
+ mutation.delete(schemaTable, timestamp);
+ mutation.delete(SystemKeyspace.BUILT_INDEXES, timestamp);
+ return mutation;
+ }
+
+ private static KSMetaData createKeyspaceFromSchemaPartitions(Row serializedKeyspace, Row serializedTables, Row serializedTypes)
+ {
+ Collection<CFMetaData> tables = createTablesFromTablesPartition(serializedTables).values();
+ UTMetaData types = new UTMetaData(createTypesFromPartition(serializedTypes));
+ return createKeyspaceFromSchemaPartition(serializedKeyspace).cloneWith(tables, types);
+ }
+
+ public static KSMetaData createKeyspaceFromName(String keyspace)
+ {
+ Row partition = readSchemaPartitionForKeyspace(KEYSPACES, keyspace);
+
+ if (isEmptySchemaPartition(partition))
+ throw new RuntimeException(String.format("%s not found in the schema definitions keyspaceName (%s).", keyspace, KEYSPACES));
+
+ return createKeyspaceFromSchemaPartition(partition);
+ }
+
+ /**
+ * Deserialize only Keyspace attributes without nested tables or types
+ *
+ * @param partition Keyspace attributes in serialized form
+ */
+ private static KSMetaData createKeyspaceFromSchemaPartition(Row partition)
+ {
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, KEYSPACES);
+ UntypedResultSet.Row row = QueryProcessor.resultify(query, partition).one();
+ try
+ {
+ return new KSMetaData(row.getString("keyspace_name"),
+ AbstractReplicationStrategy.getClass(row.getString("strategy_class")),
+ fromJsonMap(row.getString("strategy_options")),
+ row.getBoolean("durable_writes"));
+ }
+ catch (ConfigurationException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+
+ /*
+ * User type metadata serialization/deserialization.
+ */
+
+ public static Mutation makeCreateTypeMutation(KSMetaData keyspace, UserType type, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+ addTypeToSchemaMutation(type, timestamp, mutation);
+ return mutation;
+ }
+
+ private static void addTypeToSchemaMutation(UserType type, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Usertypes);
+
+ Composite prefix = Usertypes.comparator.make(type.name);
+ CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+ adder.resetCollection("field_names");
+ adder.resetCollection("field_types");
+
+ for (int i = 0; i < type.size(); i++)
+ {
+ adder.addListEntry("field_names", type.fieldName(i));
+ adder.addListEntry("field_types", type.fieldType(i).toString());
+ }
+ }
+
+ public static Mutation dropTypeFromSchemaMutation(KSMetaData keyspace, UserType type, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+ ColumnFamily cells = mutation.addOrGet(Usertypes);
+ int ldt = (int) (System.currentTimeMillis() / 1000);
+
+ Composite prefix = Usertypes.comparator.make(type.name);
+ cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+ return mutation;
+ }
+
+ private static Map<ByteBuffer, UserType> createTypesFromPartition(Row partition)
+ {
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, USERTYPES);
+ Map<ByteBuffer, UserType> types = new HashMap<>();
+ for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+ {
+ UserType type = createTypeFromRow(row);
+ types.put(type.name, type);
+ }
+ return types;
+ }
+
+ private static UserType createTypeFromRow(UntypedResultSet.Row row)
+ {
+ String keyspace = row.getString("keyspace_name");
+ ByteBuffer name = ByteBufferUtil.bytes(row.getString("type_name"));
+ List<String> rawColumns = row.getList("field_names", UTF8Type.instance);
+ List<String> rawTypes = row.getList("field_types", UTF8Type.instance);
+
+ List<ByteBuffer> columns = new ArrayList<>(rawColumns.size());
+ for (String rawColumn : rawColumns)
+ columns.add(ByteBufferUtil.bytes(rawColumn));
+
+ List<AbstractType<?>> types = new ArrayList<>(rawTypes.size());
+ for (String rawType : rawTypes)
+ types.add(parseType(rawType));
+
+ return new UserType(keyspace, name, columns, types);
+ }
+
+ /*
+ * Table metadata serialization/deserialization.
+ */
+
+ public static Mutation makeCreateTableMutation(KSMetaData keyspace, CFMetaData table, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+ addTableToSchemaMutation(table, timestamp, true, mutation);
+ return mutation;
+ }
+
+ private static void addTableToSchemaMutation(CFMetaData table, long timestamp, boolean withColumnsAndTriggers, Mutation mutation)
+ {
+ // For property that can be null (and can be changed), we insert tombstones, to make sure
+ // we don't keep a property the user has removed
+ ColumnFamily cells = mutation.addOrGet(Columnfamilies);
+ Composite prefix = Columnfamilies.comparator.make(table.cfName);
+ CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+ adder.add("cf_id", table.cfId);
+ adder.add("type", table.cfType.toString());
+
+ if (table.isSuper())
+ {
+ // We need to continue saving the comparator and subcomparator separatly, otherwise
+ // we won't know at deserialization if the subcomparator should be taken into account
+ // TODO: we should implement an on-start migration if we want to get rid of that.
+ adder.add("comparator", table.comparator.subtype(0).toString());
+ adder.add("subcomparator", table.comparator.subtype(1).toString());
+ }
+ else
+ {
+ adder.add("comparator", table.comparator.toString());
+ }
+
+ adder.add("bloom_filter_fp_chance", table.getBloomFilterFpChance());
+ adder.add("caching", table.getCaching().toString());
+ adder.add("comment", table.getComment());
+ adder.add("compaction_strategy_class", table.compactionStrategyClass.getName());
+ adder.add("compaction_strategy_options", json(table.compactionStrategyOptions));
+ adder.add("compression_parameters", json(table.compressionParameters.asThriftOptions()));
+ adder.add("default_time_to_live", table.getDefaultTimeToLive());
+ adder.add("default_validator", table.getDefaultValidator().toString());
+ adder.add("gc_grace_seconds", table.getGcGraceSeconds());
+ adder.add("key_validator", table.getKeyValidator().toString());
+ adder.add("local_read_repair_chance", table.getDcLocalReadRepairChance());
+ adder.add("max_compaction_threshold", table.getMaxCompactionThreshold());
+ adder.add("max_index_interval", table.getMaxIndexInterval());
+ adder.add("memtable_flush_period_in_ms", table.getMemtableFlushPeriod());
+ adder.add("min_compaction_threshold", table.getMinCompactionThreshold());
+ adder.add("min_index_interval", table.getMinIndexInterval());
+ adder.add("read_repair_chance", table.getReadRepairChance());
+ adder.add("speculative_retry", table.getSpeculativeRetry().toString());
+
+ for (Map.Entry<ColumnIdentifier, Long> entry : table.getDroppedColumns().entrySet())
+ adder.addMapEntry("dropped_columns", entry.getKey().toString(), entry.getValue());
+
+ adder.add("is_dense", table.getIsDense());
+
+ if (withColumnsAndTriggers)
+ {
+ for (ColumnDefinition column : table.allColumns())
+ addColumnToSchemaMutation(table, column, timestamp, mutation);
+
+ for (TriggerDefinition trigger : table.getTriggers().values())
+ addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
+ }
+ }
+
+ public static Mutation makeUpdateTableMutation(KSMetaData keyspace,
+ CFMetaData oldTable,
+ CFMetaData newTable,
+ long timestamp,
+ boolean fromThrift)
+ {
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+ addTableToSchemaMutation(newTable, timestamp, false, mutation);
+
+ MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldTable.getColumnMetadata(),
+ newTable.getColumnMetadata());
+
+ // columns that are no longer needed
+ for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
+ {
+ // Thrift only knows about the REGULAR ColumnDefinition type, so don't consider other type
+ // are being deleted just because they are not here.
+ if (fromThrift && column.kind != ColumnDefinition.Kind.REGULAR)
+ continue;
+
+ dropColumnFromSchemaMutation(oldTable, column, timestamp, mutation);
+ }
+
+ // newly added columns
+ for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
+ addColumnToSchemaMutation(newTable, column, timestamp, mutation);
+
+ // old columns with updated attributes
+ for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
+ addColumnToSchemaMutation(newTable, newTable.getColumnDefinition(name), timestamp, mutation);
+
+ MapDifference<String, TriggerDefinition> triggerDiff = Maps.difference(oldTable.getTriggers(), newTable.getTriggers());
+
+ // dropped triggers
+ for (TriggerDefinition trigger : triggerDiff.entriesOnlyOnLeft().values())
+ dropTriggerFromSchemaMutation(oldTable, trigger, timestamp, mutation);
+
+ // newly created triggers
+ for (TriggerDefinition trigger : triggerDiff.entriesOnlyOnRight().values())
+ addTriggerToSchemaMutation(newTable, trigger, timestamp, mutation);
+
+ return mutation;
+ }
+
+ public static Mutation makeDropTableMutation(KSMetaData keyspace, CFMetaData table, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+ ColumnFamily cells = mutation.addOrGet(Columnfamilies);
+ int ldt = (int) (System.currentTimeMillis() / 1000);
+
+ Composite prefix = Columnfamilies.comparator.make(table.cfName);
+ cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+ for (ColumnDefinition column : table.allColumns())
+ dropColumnFromSchemaMutation(table, column, timestamp, mutation);
+
+ for (TriggerDefinition trigger : table.getTriggers().values())
+ dropTriggerFromSchemaMutation(table, trigger, timestamp, mutation);
+
+ // TODO: get rid of in #6717
+ ColumnFamily indexCells = mutation.addOrGet(SystemKeyspace.BuiltIndexes);
+ for (String indexName : Keyspace.open(keyspace.name).getColumnFamilyStore(table.cfName).getBuiltIndexes())
+ indexCells.addTombstone(indexCells.getComparator().makeCellName(indexName), ldt, timestamp);
+
+ return mutation;
+ }
+
+ public static CFMetaData createTableFromName(String keyspace, String table)
+ {
+ Row partition = readSchemaPartitionForTable(COLUMNFAMILIES, keyspace, table);
+
+ if (isEmptySchemaPartition(partition))
+ throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspace, table));
+
+ return createTableFromTablePartition(partition);
+ }
+
+ /**
+ * Deserialize tables from low-level schema representation, all of them belong to the same keyspace
+ *
+ * @return map containing name of the table and its metadata for faster lookup
+ */
+ private static Map<String, CFMetaData> createTablesFromTablesPartition(Row partition)
+ {
+ if (partition.cf == null)
+ return Collections.emptyMap();
+
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+ Map<String, CFMetaData> tables = new HashMap<>();
+ for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+ {
+ CFMetaData cfm = createTableFromTableRow(row);
+ tables.put(cfm.cfName, cfm);
+ }
+ return tables;
+ }
+
+ public static CFMetaData createTableFromTablePartitionAndColumnsPartition(Row serializedTable, Row serializedColumns)
+ {
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+ return createTableFromTableRowAndColumnsPartition(QueryProcessor.resultify(query, serializedTable).one(), serializedColumns);
+ }
+
+ private static CFMetaData createTableFromTableRowAndColumnsPartition(UntypedResultSet.Row tableRow, Row serializedColumns)
+ {
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNS);
+ return createTableFromTableRowAndColumnRows(tableRow, QueryProcessor.resultify(query, serializedColumns));
+ }
+
+ private static CFMetaData createTableFromTablePartition(Row row)
+ {
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, COLUMNFAMILIES);
+ return createTableFromTableRow(QueryProcessor.resultify(query, row).one());
+ }
+
+ /**
+ * Deserialize table metadata from low-level representation
+ *
+ * @return Metadata deserialized from schema
+ */
+ private static CFMetaData createTableFromTableRow(UntypedResultSet.Row result)
+ {
+ String ksName = result.getString("keyspace_name");
+ String cfName = result.getString("columnfamily_name");
+
+ Row serializedColumns = readSchemaPartitionForTable(COLUMNS, ksName, cfName);
+ CFMetaData cfm = createTableFromTableRowAndColumnsPartition(result, serializedColumns);
+
+ Row serializedTriggers = readSchemaPartitionForTable(TRIGGERS, ksName, cfName);
+ try
+ {
+ for (TriggerDefinition trigger : createTriggersFromTriggersPartition(serializedTriggers))
+ cfm.addTriggerDefinition(trigger);
+ }
+ catch (InvalidRequestException e)
+ {
+ throw new RuntimeException(e);
+ }
+
+ return cfm;
+ }
+
+ public static CFMetaData createTableFromTableRowAndColumnRows(UntypedResultSet.Row result,
+ UntypedResultSet serializedColumnDefinitions)
+ {
+ try
+ {
+ String ksName = result.getString("keyspace_name");
+ String cfName = result.getString("columnfamily_name");
+
+ AbstractType<?> rawComparator = TypeParser.parse(result.getString("comparator"));
+ AbstractType<?> subComparator = result.has("subcomparator") ? TypeParser.parse(result.getString("subcomparator")) : null;
+ ColumnFamilyType cfType = ColumnFamilyType.valueOf(result.getString("type"));
+
+ AbstractType<?> fullRawComparator = CFMetaData.makeRawAbstractType(rawComparator, subComparator);
+
+ List<ColumnDefinition> columnDefs = createColumnsFromColumnRows(serializedColumnDefinitions,
+ ksName,
+ cfName,
+ fullRawComparator,
+ cfType == ColumnFamilyType.Super);
+
+ boolean isDense = result.has("is_dense")
+ ? result.getBoolean("is_dense")
+ : CFMetaData.calculateIsDense(fullRawComparator, columnDefs);
+
+ CellNameType comparator = CellNames.fromAbstractType(fullRawComparator, isDense);
+
+ // if we are upgrading, we use id generated from names initially
+ UUID cfId = result.has("cf_id")
+ ? result.getUUID("cf_id")
+ : CFMetaData.generateLegacyCfId(ksName, cfName);
+
+ CFMetaData cfm = new CFMetaData(ksName, cfName, cfType, comparator, cfId);
+ cfm.isDense(isDense);
+
+ cfm.readRepairChance(result.getDouble("read_repair_chance"));
+ cfm.dcLocalReadRepairChance(result.getDouble("local_read_repair_chance"));
+ cfm.gcGraceSeconds(result.getInt("gc_grace_seconds"));
+ cfm.defaultValidator(TypeParser.parse(result.getString("default_validator")));
+ cfm.keyValidator(TypeParser.parse(result.getString("key_validator")));
+ cfm.minCompactionThreshold(result.getInt("min_compaction_threshold"));
+ cfm.maxCompactionThreshold(result.getInt("max_compaction_threshold"));
+ if (result.has("comment"))
+ cfm.comment(result.getString("comment"));
+ if (result.has("memtable_flush_period_in_ms"))
+ cfm.memtableFlushPeriod(result.getInt("memtable_flush_period_in_ms"));
+ cfm.caching(CachingOptions.fromString(result.getString("caching")));
+ if (result.has("default_time_to_live"))
+ cfm.defaultTimeToLive(result.getInt("default_time_to_live"));
+ if (result.has("speculative_retry"))
+ cfm.speculativeRetry(CFMetaData.SpeculativeRetry.fromString(result.getString("speculative_retry")));
+ cfm.compactionStrategyClass(CFMetaData.createCompactionStrategy(result.getString("compaction_strategy_class")));
+ cfm.compressionParameters(CompressionParameters.create(fromJsonMap(result.getString("compression_parameters"))));
+ cfm.compactionStrategyOptions(fromJsonMap(result.getString("compaction_strategy_options")));
+
+ if (result.has("min_index_interval"))
+ cfm.minIndexInterval(result.getInt("min_index_interval"));
+
+ if (result.has("max_index_interval"))
+ cfm.maxIndexInterval(result.getInt("max_index_interval"));
+
+ if (result.has("bloom_filter_fp_chance"))
+ cfm.bloomFilterFpChance(result.getDouble("bloom_filter_fp_chance"));
+ else
+ cfm.bloomFilterFpChance(cfm.getBloomFilterFpChance());
+
+ if (result.has("dropped_columns"))
+ cfm.droppedColumns(convertDroppedColumns(result.getMap("dropped_columns", UTF8Type.instance, LongType.instance)));
+
+ for (ColumnDefinition cd : columnDefs)
+ cfm.addOrReplaceColumnDefinition(cd);
+
+ return cfm.rebuild();
+ }
+ catch (SyntaxException | ConfigurationException e)
+ {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private static Map<ColumnIdentifier, Long> convertDroppedColumns(Map<String, Long> raw)
+ {
+ Map<ColumnIdentifier, Long> converted = Maps.newHashMap();
+ for (Map.Entry<String, Long> entry : raw.entrySet())
+ converted.put(new ColumnIdentifier(entry.getKey(), true), entry.getValue());
+ return converted;
+ }
+
+ /*
+ * Column metadata serialization/deserialization.
+ */
+
+ private static void addColumnToSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Columns);
+ Composite prefix = Columns.comparator.make(table.cfName, column.name.toString());
+ CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+ adder.add("validator", column.type.toString());
+ adder.add("type", serializeKind(column.kind));
+ adder.add("component_index", column.isOnAllComponents() ? null : column.position());
+ adder.add("index_name", column.getIndexName());
+ adder.add("index_type", column.getIndexType() == null ? null : column.getIndexType().toString());
+ adder.add("index_options", json(column.getIndexOptions()));
+ }
+
+ private static String serializeKind(ColumnDefinition.Kind kind)
+ {
+ // For backward compatibility we need to special case CLUSTERING_COLUMN
+ return kind == ColumnDefinition.Kind.CLUSTERING_COLUMN ? "clustering_key" : kind.toString().toLowerCase();
+ }
+
+ private static ColumnDefinition.Kind deserializeKind(String kind)
+ {
+ if (kind.equalsIgnoreCase("clustering_key"))
+ return ColumnDefinition.Kind.CLUSTERING_COLUMN;
+ return Enum.valueOf(ColumnDefinition.Kind.class, kind.toUpperCase());
+ }
+
+ private static void dropColumnFromSchemaMutation(CFMetaData table, ColumnDefinition column, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Columns);
+ int ldt = (int) (System.currentTimeMillis() / 1000);
+
+ // Note: we do want to use name.toString(), not name.bytes directly for backward compatibility (For CQL3, this won't make a difference).
+ Composite prefix = Columns.comparator.make(table.cfName, column.name.toString());
+ cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+ }
+
+ private static List<ColumnDefinition> createColumnsFromColumnRows(UntypedResultSet rows,
+ String keyspace,
+ String table,
+ AbstractType<?> rawComparator,
+ boolean isSuper)
+ {
+ List<ColumnDefinition> columns = new ArrayList<>();
+ for (UntypedResultSet.Row row : rows)
+ columns.add(createColumnFromColumnRow(row, keyspace, table, rawComparator, isSuper));
+ return columns;
+ }
+
+ private static ColumnDefinition createColumnFromColumnRow(UntypedResultSet.Row row,
+ String keyspace,
+ String table,
+ AbstractType<?> rawComparator,
+ boolean isSuper)
+ {
+ ColumnDefinition.Kind kind = deserializeKind(row.getString("type"));
+
+ Integer componentIndex = null;
+ if (row.has("component_index"))
+ componentIndex = row.getInt("component_index");
+ else if (kind == ColumnDefinition.Kind.CLUSTERING_COLUMN && isSuper)
+ componentIndex = 1; // A ColumnDefinition for super columns applies to the column component
+
+ // Note: we save the column name as string, but we should not assume that it is an UTF8 name, we
+ // we need to use the comparator fromString method
+ AbstractType<?> comparator = kind == ColumnDefinition.Kind.REGULAR
+ ? getComponentComparator(rawComparator, componentIndex)
+ : UTF8Type.instance;
+ ColumnIdentifier name = new ColumnIdentifier(comparator.fromString(row.getString("column_name")), comparator);
+
+ AbstractType<?> validator = parseType(row.getString("validator"));
+
+ IndexType indexType = null;
+ if (row.has("index_type"))
+ indexType = IndexType.valueOf(row.getString("index_type"));
+
+ Map<String, String> indexOptions = null;
+ if (row.has("index_options"))
+ indexOptions = fromJsonMap(row.getString("index_options"));
+
+ String indexName = null;
+ if (row.has("index_name"))
+ indexName = row.getString("index_name");
+
+ return new ColumnDefinition(keyspace, table, name, validator, indexType, indexOptions, indexName, componentIndex, kind);
+ }
+
+ private static AbstractType<?> getComponentComparator(AbstractType<?> rawComparator, Integer componentIndex)
+ {
+ return (componentIndex == null || (componentIndex == 0 && !(rawComparator instanceof CompositeType)))
+ ? rawComparator
+ : ((CompositeType)rawComparator).types.get(componentIndex);
+ }
+
+ /*
+ * Trigger metadata serialization/deserialization.
+ */
+
+ private static void addTriggerToSchemaMutation(CFMetaData table, TriggerDefinition trigger, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Triggers);
+ Composite prefix = Triggers.comparator.make(table.cfName, trigger.name);
+ CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+ adder.addMapEntry("trigger_options", "class", trigger.classOption);
+ }
+
+ private static void dropTriggerFromSchemaMutation(CFMetaData table, TriggerDefinition trigger, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Triggers);
+ int ldt = (int) (System.currentTimeMillis() / 1000);
+
+ Composite prefix = Triggers.comparator.make(table.cfName, trigger.name);
+ cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+ }
+
+ /**
+ * Deserialize triggers from storage-level representation.
+ *
+ * @param partition storage-level partition containing the trigger definitions
+ * @return the list of processed TriggerDefinitions
+ */
+ private static List<TriggerDefinition> createTriggersFromTriggersPartition(Row partition)
+ {
+ List<TriggerDefinition> triggers = new ArrayList<>();
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, TRIGGERS);
+ for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+ {
+ String name = row.getString("trigger_name");
+ String classOption = row.getMap("trigger_options", UTF8Type.instance, UTF8Type.instance).get("class");
+ triggers.add(new TriggerDefinition(name, classOption));
+ }
+ return triggers;
+ }
+
+ /*
+ * UDF metadata serialization/deserialization.
+ */
+
+ public static Mutation makeCreateFunctionMutation(KSMetaData keyspace, UDFunction function, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+ addFunctionToSchemaMutation(function, timestamp, mutation);
+ return mutation;
+ }
+
+ private static void addFunctionToSchemaMutation(UDFunction function, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Functions);
+ Composite prefix = Functions.comparator.make(function.name().name, UDHelper.calculateSignature(function));
+ CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+ adder.resetCollection("argument_names");
+ adder.resetCollection("argument_types");
+
+ for (int i = 0; i < function.argNames().size(); i++)
+ {
+ adder.addListEntry("argument_names", function.argNames().get(i).bytes);
+ adder.addListEntry("argument_types", function.argTypes().get(i).toString());
+ }
+
+ adder.add("body", function.body());
+ adder.add("is_deterministic", function.isDeterministic());
+ adder.add("language", function.language());
+ adder.add("return_type", function.returnType().toString());
+ }
+
+ public static Mutation makeDropFunctionMutation(KSMetaData keyspace, UDFunction function, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+ ColumnFamily cells = mutation.addOrGet(Functions);
+ int ldt = (int) (System.currentTimeMillis() / 1000);
+
+ Composite prefix = Functions.comparator.make(function.name().name, UDHelper.calculateSignature(function));
+ cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+ return mutation;
+ }
+
+ private static Map<ByteBuffer, UDFunction> createFunctionsFromFunctionsPartition(Row partition)
+ {
+ Map<ByteBuffer, UDFunction> functions = new HashMap<>();
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, FUNCTIONS);
+ for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+ {
+ UDFunction function = createFunctionFromFunctionRow(row);
+ functions.put(UDHelper.calculateSignature(function), function);
+ }
+ return functions;
+ }
+
+ private static UDFunction createFunctionFromFunctionRow(UntypedResultSet.Row row)
+ {
+ String ksName = row.getString("keyspace_name");
+ String functionName = row.getString("function_name");
+ FunctionName name = new FunctionName(ksName, functionName);
+
+ List<ColumnIdentifier> argNames = new ArrayList<>();
+ if (row.has("argument_names"))
+ for (String arg : row.getList("argument_names", UTF8Type.instance))
+ argNames.add(new ColumnIdentifier(arg, true));
+
+ List<AbstractType<?>> argTypes = new ArrayList<>();
+ if (row.has("argument_types"))
+ for (String type : row.getList("argument_types", UTF8Type.instance))
+ argTypes.add(parseType(type));
+
+ AbstractType<?> returnType = parseType(row.getString("return_type"));
+
+ boolean isDeterministic = row.getBoolean("is_deterministic");
+ String language = row.getString("language");
+ String body = row.getString("body");
+
+ try
+ {
+ return UDFunction.create(name, argNames, argTypes, returnType, language, body, isDeterministic);
+ }
+ catch (InvalidRequestException e)
+ {
+ logger.error(String.format("Cannot load function '%s' from schema: this function won't be available (on this node)", name), e);
+ return UDFunction.createBrokenFunction(name, argNames, argTypes, returnType, language, body, e);
+ }
+ }
+
+ /*
+ * Aggregate UDF metadata serialization/deserialization.
+ */
+
+ public static Mutation makeCreateAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+ addAggregateToSchemaMutation(aggregate, timestamp, mutation);
+ return mutation;
+ }
+
+ private static void addAggregateToSchemaMutation(UDAggregate aggregate, long timestamp, Mutation mutation)
+ {
+ ColumnFamily cells = mutation.addOrGet(Aggregates);
+ Composite prefix = Aggregates.comparator.make(aggregate.name().name, UDHelper.calculateSignature(aggregate));
+ CFRowAdder adder = new CFRowAdder(cells, prefix, timestamp);
+
+ adder.resetCollection("argument_types");
+ adder.add("return_type", aggregate.returnType().toString());
+ adder.add("state_func", aggregate.stateFunction().name().name);
+ if (aggregate.stateType() != null)
+ adder.add("state_type", aggregate.stateType().toString());
+ if (aggregate.finalFunction() != null)
+ adder.add("final_func", aggregate.finalFunction().name().name);
+ if (aggregate.initialCondition() != null)
+ adder.add("initcond", aggregate.initialCondition());
+
+ for (AbstractType<?> argType : aggregate.argTypes())
+ adder.addListEntry("argument_types", argType.toString());
+ }
+
+ private static Map<ByteBuffer, UDAggregate> createAggregatesFromAggregatesPartition(Row partition)
+ {
+ Map<ByteBuffer, UDAggregate> aggregates = new HashMap<>();
+ String query = String.format("SELECT * FROM %s.%s", SystemKeyspace.NAME, AGGREGATES);
+ for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
+ {
+ UDAggregate aggregate = createAggregateFromAggregateRow(row);
+ aggregates.put(UDHelper.calculateSignature(aggregate), aggregate);
+ }
+ return aggregates;
+ }
+
+ private static UDAggregate createAggregateFromAggregateRow(UntypedResultSet.Row row)
+ {
+ String ksName = row.getString("keyspace_name");
+ String functionName = row.getString("aggregate_name");
+ FunctionName name = new FunctionName(ksName, functionName);
+
+ List<String> types = row.getList("argument_types", UTF8Type.instance);
+
+ List<AbstractType<?>> argTypes;
+ if (types == null)
+ {
+ argTypes = Collections.emptyList();
+ }
+ else
+ {
+ argTypes = new ArrayList<>(types.size());
+ for (String type : types)
+ argTypes.add(parseType(type));
+ }
+
+ AbstractType<?> returnType = parseType(row.getString("return_type"));
+
+ FunctionName stateFunc = new FunctionName(ksName, row.getString("state_func"));
+ FunctionName finalFunc = row.has("final_func") ? new FunctionName(ksName, row.getString("final_func")) : null;
+ AbstractType<?> stateType = row.has("state_type") ? parseType(row.getString("state_type")) : null;
+ ByteBuffer initcond = row.has("initcond") ? row.getBytes("initcond") : null;
+
+ try
+ {
+ return UDAggregate.create(name, argTypes, returnType, stateFunc, finalFunc, stateType, initcond);
+ }
+ catch (InvalidRequestException reason)
+ {
+ return UDAggregate.createBroken(name, argTypes, returnType, initcond, reason);
+ }
+ }
+
+ public static Mutation makeDropAggregateMutation(KSMetaData keyspace, UDAggregate aggregate, long timestamp)
+ {
+ // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+ Mutation mutation = makeCreateKeyspaceMutation(keyspace, timestamp, false);
+
+ ColumnFamily cells = mutation.addOrGet(Aggregates);
+ int ldt = (int) (System.currentTimeMillis() / 1000);
+
+ Composite prefix = Aggregates.comparator.make(aggregate.name().name, UDHelper.calculateSignature(aggregate));
+ cells.addAtom(new RangeTombstone(prefix, prefix.end(), timestamp, ldt));
+
+ return mutation;
+ }
+
+ private static AbstractType<?> parseType(String str)
+ {
+ try
+ {
+ return TypeParser.parse(str);
+ }
+ catch (SyntaxException | ConfigurationException e)
+ {
+ // We only use this when reading the schema where we shouldn't get an error
+ throw new RuntimeException(e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 99a2d87..edc91ab 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -34,7 +34,6 @@ import javax.management.StandardMBean;
import com.google.common.collect.Iterables;
import com.google.common.util.concurrent.Uninterruptibles;
-import org.hyperic.sigar.SigarException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -46,7 +45,6 @@ import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.functions.Functions;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Directories;
import org.apache.cassandra.db.Keyspace;
@@ -253,10 +251,8 @@ public class CassandraDaemon
exitOrFail(100, "Fatal exception during initialization", e);
}
-
- // load keyspace && function descriptions.
- DatabaseDescriptor.loadSchemas();
- Functions.loadUDFFromSchema();
+ // load schema from disk
+ Schema.instance.loadFromDisk();
// clean up compaction leftovers
Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = SystemKeyspace.getUnfinishedCompactions();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/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 9c8d7d8..dcdf838 100644
--- a/src/java/org/apache/cassandra/service/ClientState.java
+++ b/src/java/org/apache/cassandra/service/ClientState.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.QueryHandler;
import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.exceptions.AuthenticationException;
import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -58,7 +59,7 @@ public class ClientState
{
// We want these system cfs to be always readable to authenticated users since many tools rely on them
// (nodetool, cqlsh, bulkloader, etc.)
- for (String cf : Iterables.concat(Arrays.asList(SystemKeyspace.LOCAL_TABLE, SystemKeyspace.PEERS_TABLE), SystemKeyspace.ALL_SCHEMA_TABLES))
+ for (String cf : Iterables.concat(Arrays.asList(SystemKeyspace.LOCAL, SystemKeyspace.PEERS), LegacySchemaTables.ALL))
READABLE_SYSTEM_RESOURCES.add(DataResource.columnFamily(SystemKeyspace.NAME, cf));
PROTECTED_AUTH_RESOURCES.addAll(DatabaseDescriptor.getAuthenticator().protectedResources());
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index c3fe1fa..fe32559 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -37,7 +37,6 @@ import org.apache.cassandra.concurrent.Stage;
import org.apache.cassandra.concurrent.StageManager;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.UTMetaData;
import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.functions.UDAggregate;
import org.apache.cassandra.cql3.functions.UDFunction;
@@ -50,6 +49,7 @@ import org.apache.cassandra.io.IVersionedSerializer;
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.net.MessageOut;
import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.WrappedRunnable;
@@ -63,7 +63,7 @@ public class MigrationManager
public static final int MIGRATION_DELAY_IN_MS = 60000;
- private final List<IMigrationListener> listeners = new CopyOnWriteArrayList<IMigrationListener>();
+ private final List<IMigrationListener> listeners = new CopyOnWriteArrayList<>();
private MigrationManager() {}
@@ -182,52 +182,40 @@ public class MigrationManager
listener.onCreateFunction(udf.name().keyspace, udf.name().name);
}
- public void notifyUpdateFunction(UDFunction udf)
- {
- for (IMigrationListener listener : listeners)
- listener.onUpdateFunction(udf.name().keyspace, udf.name().name);
- }
-
- public void notifyDropFunction(UDFunction udf)
- {
- for (IMigrationListener listener : listeners)
- listener.onDropFunction(udf.name().keyspace, udf.name().name);
- }
-
public void notifyCreateAggregate(UDAggregate udf)
{
for (IMigrationListener listener : listeners)
listener.onCreateAggregate(udf.name().keyspace, udf.name().name);
}
- public void notifyUpdateAggregate(UDAggregate udf)
+ public void notifyUpdateKeyspace(KSMetaData ksm)
{
for (IMigrationListener listener : listeners)
- listener.onUpdateAggregate(udf.name().keyspace, udf.name().name);
+ listener.onUpdateKeyspace(ksm.name);
}
- public void notifyDropAggregate(UDAggregate udf)
+ public void notifyUpdateColumnFamily(CFMetaData cfm)
{
for (IMigrationListener listener : listeners)
- listener.onDropAggregate(udf.name().keyspace, udf.name().name);
+ listener.onUpdateColumnFamily(cfm.ksName, cfm.cfName);
}
- public void notifyUpdateKeyspace(KSMetaData ksm)
+ public void notifyUpdateUserType(UserType ut)
{
for (IMigrationListener listener : listeners)
- listener.onUpdateKeyspace(ksm.name);
+ listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
}
- public void notifyUpdateColumnFamily(CFMetaData cfm)
+ public void notifyUpdateFunction(UDFunction udf)
{
for (IMigrationListener listener : listeners)
- listener.onUpdateColumnFamily(cfm.ksName, cfm.cfName);
+ listener.onUpdateFunction(udf.name().keyspace, udf.name().name);
}
- public void notifyUpdateUserType(UserType ut)
+ public void notifyUpdateAggregate(UDAggregate udf)
{
for (IMigrationListener listener : listeners)
- listener.onUpdateUserType(ut.keyspace, ut.getNameAsString());
+ listener.onUpdateAggregate(udf.name().keyspace, udf.name().name);
}
public void notifyDropKeyspace(KSMetaData ksm)
@@ -248,6 +236,18 @@ public class MigrationManager
listener.onDropUserType(ut.keyspace, ut.getNameAsString());
}
+ public void notifyDropFunction(UDFunction udf)
+ {
+ for (IMigrationListener listener : listeners)
+ listener.onDropFunction(udf.name().keyspace, udf.name().name);
+ }
+
+ public void notifyDropAggregate(UDAggregate udf)
+ {
+ for (IMigrationListener listener : listeners)
+ listener.onDropAggregate(udf.name().keyspace, udf.name().name);
+ }
+
public static void announceNewKeyspace(KSMetaData ksm) throws ConfigurationException
{
announceNewKeyspace(ksm, false);
@@ -266,7 +266,7 @@ public class MigrationManager
throw new AlreadyExistsException(ksm.name);
logger.info(String.format("Create new Keyspace: %s", ksm));
- announce(ksm.toSchema(timestamp), announceLocally);
+ announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, timestamp), announceLocally);
}
public static void announceNewColumnFamily(CFMetaData cfm) throws ConfigurationException
@@ -285,17 +285,27 @@ public class MigrationManager
throw new AlreadyExistsException(cfm.ksName, cfm.cfName);
logger.info(String.format("Create new table: %s", cfm));
- announce(addSerializedKeyspace(cfm.toSchema(FBUtilities.timestampMicros()), cfm.ksName), announceLocally);
+ announce(LegacySchemaTables.makeCreateTableMutation(ksm, cfm, FBUtilities.timestampMicros()), announceLocally);
}
- public static void announceNewType(UserType newType)
+ public static void announceNewType(UserType newType, boolean announceLocally)
{
- announceNewType(newType, false);
+ KSMetaData ksm = Schema.instance.getKSMetaData(newType.keyspace);
+ announce(LegacySchemaTables.makeCreateTypeMutation(ksm, newType, FBUtilities.timestampMicros()), announceLocally);
}
- public static void announceNewType(UserType newType, boolean announceLocally)
+ public static void announceNewFunction(UDFunction udf, boolean announceLocally)
{
- announce(addSerializedKeyspace(UTMetaData.toSchema(newType, FBUtilities.timestampMicros()), newType.keyspace), announceLocally);
+ logger.info(String.format("Create scalar function '%s'", udf.name()));
+ KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+ announce(LegacySchemaTables.makeCreateFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
+ }
+
+ public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
+ {
+ logger.info(String.format("Create aggregate function '%s'", udf.name()));
+ KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+ announce(LegacySchemaTables.makeCreateAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
}
public static void announceKeyspaceUpdate(KSMetaData ksm) throws ConfigurationException
@@ -312,7 +322,7 @@ public class MigrationManager
throw new ConfigurationException(String.format("Cannot update non existing keyspace '%s'.", ksm.name));
logger.info(String.format("Update Keyspace '%s' From %s To %s", ksm.name, oldKsm, ksm));
- announce(oldKsm.toSchemaUpdate(ksm, FBUtilities.timestampMicros()), announceLocally);
+ announce(LegacySchemaTables.makeCreateKeyspaceMutation(ksm, FBUtilities.timestampMicros()), announceLocally);
}
public static void announceColumnFamilyUpdate(CFMetaData cfm, boolean fromThrift) throws ConfigurationException
@@ -327,16 +337,12 @@ public class MigrationManager
CFMetaData oldCfm = Schema.instance.getCFMetaData(cfm.ksName, cfm.cfName);
if (oldCfm == null)
throw new ConfigurationException(String.format("Cannot update non existing table '%s' in keyspace '%s'.", cfm.cfName, cfm.ksName));
+ KSMetaData ksm = Schema.instance.getKSMetaData(cfm.ksName);
oldCfm.validateCompatility(cfm);
logger.info(String.format("Update table '%s/%s' From %s To %s", cfm.ksName, cfm.cfName, oldCfm, cfm));
- announce(addSerializedKeyspace(oldCfm.toSchemaUpdate(cfm, FBUtilities.timestampMicros(), fromThrift), cfm.ksName), announceLocally);
- }
-
- public static void announceTypeUpdate(UserType updatedType)
- {
- announceTypeUpdate(updatedType, false);
+ announce(LegacySchemaTables.makeUpdateTableMutation(ksm, oldCfm, cfm, FBUtilities.timestampMicros(), fromThrift), announceLocally);
}
public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
@@ -356,7 +362,7 @@ public class MigrationManager
throw new ConfigurationException(String.format("Cannot drop non existing keyspace '%s'.", ksName));
logger.info(String.format("Drop Keyspace '%s'", oldKsm.name));
- announce(oldKsm.dropFromSchema(FBUtilities.timestampMicros()), announceLocally);
+ announce(LegacySchemaTables.makeDropKeyspaceMutation(oldKsm, FBUtilities.timestampMicros()), announceLocally);
}
public static void announceColumnFamilyDrop(String ksName, String cfName) throws ConfigurationException
@@ -369,16 +375,10 @@ public class MigrationManager
CFMetaData oldCfm = Schema.instance.getCFMetaData(ksName, cfName);
if (oldCfm == null)
throw new ConfigurationException(String.format("Cannot drop non existing table '%s' in keyspace '%s'.", cfName, ksName));
+ KSMetaData ksm = Schema.instance.getKSMetaData(ksName);
logger.info(String.format("Drop table '%s/%s'", oldCfm.ksName, oldCfm.cfName));
- announce(addSerializedKeyspace(oldCfm.dropFromSchema(FBUtilities.timestampMicros()), ksName), announceLocally);
- }
-
- // Include the serialized keyspace for when a target node missed the CREATE KEYSPACE migration (see #5631).
- private static Mutation addSerializedKeyspace(Mutation migration, String ksName)
- {
- migration.add(SystemKeyspace.readSchemaRow(SystemKeyspace.SCHEMA_KEYSPACES_TABLE, ksName).cf);
- return migration;
+ announce(LegacySchemaTables.makeDropTableMutation(ksm, oldCfm, FBUtilities.timestampMicros()), announceLocally);
}
public static void announceTypeDrop(UserType droppedType)
@@ -388,35 +388,22 @@ public class MigrationManager
public static void announceTypeDrop(UserType droppedType, boolean announceLocally)
{
- announce(addSerializedKeyspace(UTMetaData.dropFromSchema(droppedType, FBUtilities.timestampMicros()), droppedType.keyspace), announceLocally);
+ KSMetaData ksm = Schema.instance.getKSMetaData(droppedType.keyspace);
+ announce(LegacySchemaTables.dropTypeFromSchemaMutation(ksm, droppedType, FBUtilities.timestampMicros()), announceLocally);
}
public static void announceFunctionDrop(UDFunction udf, boolean announceLocally)
{
- Mutation mutation = udf.toSchemaDrop(FBUtilities.timestampMicros());
logger.info(String.format("Drop scalar function overload '%s' args '%s'", udf.name(), udf.argTypes()));
- announce(mutation, announceLocally);
+ KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+ announce(LegacySchemaTables.makeDropFunctionMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
}
public static void announceAggregateDrop(UDAggregate udf, boolean announceLocally)
{
- Mutation mutation = udf.toSchemaDrop(FBUtilities.timestampMicros());
logger.info(String.format("Drop aggregate function overload '%s' args '%s'", udf.name(), udf.argTypes()));
- announce(mutation, announceLocally);
- }
-
- public static void announceNewFunction(UDFunction udf, boolean announceLocally)
- {
- Mutation mutation = udf.toSchemaUpdate(FBUtilities.timestampMicros());
- logger.info(String.format("Create scalar function '%s'", udf.name()));
- announce(mutation, announceLocally);
- }
-
- public static void announceNewAggregate(UDAggregate udf, boolean announceLocally)
- {
- Mutation mutation = udf.toSchemaUpdate(FBUtilities.timestampMicros());
- logger.info(String.format("Create aggregate function '%s'", udf.name()));
- announce(mutation, announceLocally);
+ KSMetaData ksm = Schema.instance.getKSMetaData(udf.name().keyspace);
+ announce(LegacySchemaTables.makeDropAggregateMutation(ksm, udf, FBUtilities.timestampMicros()), announceLocally);
}
/**
@@ -429,7 +416,7 @@ public class MigrationManager
{
try
{
- DefsTables.mergeSchemaInternal(Collections.singletonList(schema), false);
+ LegacySchemaTables.mergeSchema(Collections.singletonList(schema), false);
}
catch (IOException e)
{
@@ -457,7 +444,7 @@ public class MigrationManager
{
protected void runMayThrow() throws IOException, ConfigurationException
{
- DefsTables.mergeSchema(schema);
+ LegacySchemaTables.mergeSchema(schema);
}
});
@@ -497,9 +484,7 @@ public class MigrationManager
logger.debug("Truncating schema tables...");
- // truncate schema tables
- for (String cf : SystemKeyspace.ALL_SCHEMA_TABLES)
- SystemKeyspace.schemaCFS(cf).truncateBlocking();
+ LegacySchemaTables.truncateSchemaTables();
logger.debug("Clearing local schema keyspace definitions...");
@@ -536,7 +521,7 @@ public class MigrationManager
public Collection<Mutation> deserialize(DataInput in, int version) throws IOException
{
int count = in.readInt();
- Collection<Mutation> schema = new ArrayList<Mutation>(count);
+ Collection<Mutation> schema = new ArrayList<>(count);
for (int i = 0; i < count; i++)
schema.add(Mutation.serializer.deserialize(in, version));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/service/MigrationTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationTask.java b/src/java/org/apache/cassandra/service/MigrationTask.java
index 9fdbff4..2a67e6d 100644
--- a/src/java/org/apache/cassandra/service/MigrationTask.java
+++ b/src/java/org/apache/cassandra/service/MigrationTask.java
@@ -26,7 +26,7 @@ import org.slf4j.LoggerFactory;
import org.apache.cassandra.db.Mutation;
import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.DefsTables;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.gms.FailureDetector;
import org.apache.cassandra.net.IAsyncCallback;
import org.apache.cassandra.net.MessageIn;
@@ -63,7 +63,7 @@ class MigrationTask extends WrappedRunnable
{
try
{
- DefsTables.mergeSchema(message.payload);
+ LegacySchemaTables.mergeSchema(message.payload);
}
catch (IOException e)
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/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 9c57946..d6eb317 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -713,7 +713,7 @@ public class StorageProxy implements StorageProxyMBean
null,
WriteType.SIMPLE);
Mutation mutation = new Mutation(SystemKeyspace.NAME, UUIDType.instance.decompose(uuid));
- mutation.delete(SystemKeyspace.BATCHLOG_TABLE, FBUtilities.timestampMicros());
+ mutation.delete(SystemKeyspace.BATCHLOG, FBUtilities.timestampMicros());
MessageOut<Mutation> message = mutation.createMessage();
for (InetAddress target : endpoints)
{
[3/5] cassandra git commit: Isolate schema serializaton code
Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index 3e8b0a2..503dd7f 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -26,27 +26,20 @@ import java.util.concurrent.TimeUnit;
import javax.management.openmbean.*;
import com.google.common.base.Function;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.SetMultimap;
-import com.google.common.collect.Sets;
+import com.google.common.collect.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
import org.apache.cassandra.db.compaction.CompactionHistoryTabularData;
import org.apache.cassandra.db.commitlog.ReplayPosition;
import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
-import org.apache.cassandra.db.composites.Composite;
import org.apache.cassandra.db.filter.QueryFilter;
import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -54,6 +47,7 @@ import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.LocalStrategy;
import org.apache.cassandra.metrics.RestorableMeter;
+import org.apache.cassandra.schema.LegacySchemaTables;
import org.apache.cassandra.service.StorageService;
import org.apache.cassandra.service.paxos.Commit;
import org.apache.cassandra.service.paxos.PaxosState;
@@ -70,155 +64,21 @@ public final class SystemKeyspace
public static final String NAME = "system";
- public static final String SCHEMA_KEYSPACES_TABLE = "schema_keyspaces";
- public static final String SCHEMA_COLUMNFAMILIES_TABLE = "schema_columnfamilies";
- public static final String SCHEMA_COLUMNS_TABLE = "schema_columns";
- public static final String SCHEMA_TRIGGERS_TABLE = "schema_triggers";
- public static final String SCHEMA_USER_TYPES_TABLE = "schema_usertypes";
- public static final String SCHEMA_FUNCTIONS_TABLE = "schema_functions";
- public static final String SCHEMA_AGGREGATES_TABLE = "schema_aggregates";
-
- public static final String BUILT_INDEXES_TABLE = "IndexInfo";
- public static final String HINTS_TABLE = "hints";
- public static final String BATCHLOG_TABLE = "batchlog";
- public static final String PAXOS_TABLE = "paxos";
- public static final String LOCAL_TABLE = "local";
- public static final String PEERS_TABLE = "peers";
- public static final String PEER_EVENTS_TABLE = "peer_events";
- public static final String RANGE_XFERS_TABLE = "range_xfers";
- public static final String COMPACTION_LOG_TABLE = "compactions_in_progress";
- public static final String COMPACTION_HISTORY_TABLE = "compaction_history";
- public static final String SSTABLE_ACTIVITY_TABLE = "sstable_activity";
-
- public static final List<String> ALL_SCHEMA_TABLES =
- Arrays.asList(SCHEMA_KEYSPACES_TABLE,
- SCHEMA_COLUMNFAMILIES_TABLE,
- SCHEMA_COLUMNS_TABLE,
- SCHEMA_TRIGGERS_TABLE,
- SCHEMA_USER_TYPES_TABLE,
- SCHEMA_FUNCTIONS_TABLE,
- SCHEMA_AGGREGATES_TABLE);
-
- private static int WEEK = (int) TimeUnit.DAYS.toSeconds(7);
-
- public static final CFMetaData SchemaKeyspacesTable =
- compile(SCHEMA_KEYSPACES_TABLE, "keyspace definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "durable_writes boolean,"
- + "strategy_class text,"
- + "strategy_options text,"
- + "PRIMARY KEY ((keyspace_name))) "
- + "WITH COMPACT STORAGE")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData SchemaColumnFamiliesTable =
- compile(SCHEMA_COLUMNFAMILIES_TABLE, "table definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "columnfamily_name text,"
- + "bloom_filter_fp_chance double,"
- + "caching text,"
- + "cf_id uuid," // post-2.1 UUID cfid
- + "comment text,"
- + "compaction_strategy_class text,"
- + "compaction_strategy_options text,"
- + "comparator text,"
- + "compression_parameters text,"
- + "default_time_to_live int,"
- + "default_validator text,"
- + "dropped_columns map<text, bigint>,"
- + "gc_grace_seconds int,"
- + "is_dense boolean,"
- + "key_validator text,"
- + "local_read_repair_chance double,"
- + "max_compaction_threshold int,"
- + "max_index_interval int,"
- + "memtable_flush_period_in_ms int,"
- + "min_compaction_threshold int,"
- + "min_index_interval int,"
- + "read_repair_chance double,"
- + "speculative_retry text,"
- + "subcomparator text,"
- + "type text,"
- + "PRIMARY KEY ((keyspace_name), columnfamily_name))")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData SchemaColumnsTable =
- compile(SCHEMA_COLUMNS_TABLE, "column definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "columnfamily_name text,"
- + "column_name text,"
- + "component_index int,"
- + "index_name text,"
- + "index_options text,"
- + "index_type text,"
- + "type text,"
- + "validator text,"
- + "PRIMARY KEY ((keyspace_name), columnfamily_name, column_name))")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData SchemaTriggersTable =
- compile(SCHEMA_TRIGGERS_TABLE, "trigger definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "columnfamily_name text,"
- + "trigger_name text,"
- + "trigger_options map<text, text>,"
- + "PRIMARY KEY ((keyspace_name), columnfamily_name, trigger_name))")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData SchemaUserTypesTable =
- compile(SCHEMA_USER_TYPES_TABLE, "user defined type definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "type_name text,"
- + "field_names list<text>,"
- + "field_types list<text>,"
- + "PRIMARY KEY ((keyspace_name), type_name))")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData SchemaFunctionsTable =
- compile(SCHEMA_FUNCTIONS_TABLE, "user defined function definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "function_name text,"
- + "signature blob,"
- + "argument_names list<text>,"
- + "argument_types list<text>,"
- + "body text,"
- + "deterministic boolean,"
- + "language text,"
- + "return_type text,"
- + "PRIMARY KEY ((keyspace_name), function_name, signature))")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData SchemaAggregatesTable =
- compile(SCHEMA_AGGREGATES_TABLE, "user defined aggregate definitions",
- "CREATE TABLE %s ("
- + "keyspace_name text,"
- + "aggregate_name text,"
- + "signature blob,"
- + "argument_types list<text>,"
- + "return_type text,"
- + "state_func text,"
- + "state_type text,"
- + "final_func text,"
- + "initcond blob,"
- + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))")
- .gcGraceSeconds(WEEK);
-
- public static final CFMetaData BuiltIndexesTable =
- compile(BUILT_INDEXES_TABLE, "built column indexes",
- "CREATE TABLE \"%s\" ("
- + "table_name text,"
- + "index_name text,"
- + "PRIMARY KEY ((table_name), index_name)) "
- + "WITH COMPACT STORAGE");
-
- public static final CFMetaData HintsTable =
- compile(HINTS_TABLE, "hints awaiting delivery",
+ public static final String HINTS = "hints";
+ public static final String BATCHLOG = "batchlog";
+ public static final String PAXOS = "paxos";
+ public static final String BUILT_INDEXES = "IndexInfo";
+ public static final String LOCAL = "local";
+ public static final String PEERS = "peers";
+ public static final String PEER_EVENTS = "peer_events";
+ public static final String RANGE_XFERS = "range_xfers";
+ public static final String COMPACTIONS_IN_PROGRESS = "compactions_in_progress";
+ public static final String COMPACTION_HISTORY = "compaction_history";
+ public static final String SSTABLE_ACTIVITY = "sstable_activity";
+
+ public static final CFMetaData Hints =
+ compile(HINTS,
+ "hints awaiting delivery",
"CREATE TABLE %s ("
+ "target_id uuid,"
+ "hint_id timeuuid,"
@@ -229,8 +89,9 @@ public final class SystemKeyspace
.compactionStrategyOptions(Collections.singletonMap("enabled", "false"))
.gcGraceSeconds(0);
- public static final CFMetaData BatchlogTable =
- compile(BATCHLOG_TABLE, "batches awaiting replay",
+ public static final CFMetaData Batchlog =
+ compile(BATCHLOG,
+ "batches awaiting replay",
"CREATE TABLE %s ("
+ "id uuid,"
+ "data blob,"
@@ -240,8 +101,9 @@ public final class SystemKeyspace
.compactionStrategyOptions(Collections.singletonMap("min_threshold", "2"))
.gcGraceSeconds(0);
- private static final CFMetaData PaxosTable =
- compile(PAXOS_TABLE, "in-progress paxos proposals",
+ private static final CFMetaData Paxos =
+ compile(PAXOS,
+ "in-progress paxos proposals",
"CREATE TABLE %s ("
+ "row_key blob,"
+ "cf_id UUID,"
@@ -253,8 +115,19 @@ public final class SystemKeyspace
+ "PRIMARY KEY ((row_key), cf_id))")
.compactionStrategyClass(LeveledCompactionStrategy.class);
- private static final CFMetaData LocalTable =
- compile(LOCAL_TABLE, "information about the local node",
+ // TODO: make private
+ public static final CFMetaData BuiltIndexes =
+ compile(BUILT_INDEXES,
+ "built column indexes",
+ "CREATE TABLE \"%s\" ("
+ + "table_name text,"
+ + "index_name text,"
+ + "PRIMARY KEY ((table_name), index_name)) "
+ + "WITH COMPACT STORAGE");
+
+ private static final CFMetaData Local =
+ compile(LOCAL,
+ "information about the local node",
"CREATE TABLE %s ("
+ "key text,"
+ "bootstrapped text,"
@@ -273,8 +146,9 @@ public final class SystemKeyspace
+ "truncated_at map<uuid, blob>,"
+ "PRIMARY KEY ((key)))");
- private static final CFMetaData PeersTable =
- compile(PEERS_TABLE, "information about known peers in the cluster",
+ private static final CFMetaData Peers =
+ compile(PEERS,
+ "information about known peers in the cluster",
"CREATE TABLE %s ("
+ "peer inet,"
+ "data_center text,"
@@ -287,22 +161,25 @@ public final class SystemKeyspace
+ "tokens set<varchar>,"
+ "PRIMARY KEY ((peer)))");
- private static final CFMetaData PeerEventsTable =
- compile(PEER_EVENTS_TABLE, "events related to peers",
+ private static final CFMetaData PeerEvents =
+ compile(PEER_EVENTS,
+ "events related to peers",
"CREATE TABLE %s ("
+ "peer inet,"
+ "hints_dropped map<uuid, int>,"
+ "PRIMARY KEY ((peer)))");
- private static final CFMetaData RangeXfersTable =
- compile(RANGE_XFERS_TABLE, "ranges requested for transfer",
+ private static final CFMetaData RangeXfers =
+ compile(RANGE_XFERS,
+ "ranges requested for transfer",
"CREATE TABLE %s ("
+ "token_bytes blob,"
+ "requested_at timestamp,"
+ "PRIMARY KEY ((token_bytes)))");
- private static final CFMetaData CompactionLogTable =
- compile(COMPACTION_LOG_TABLE, "unfinished compactions",
+ private static final CFMetaData CompactionsInProgress =
+ compile(COMPACTIONS_IN_PROGRESS,
+ "unfinished compactions",
"CREATE TABLE %s ("
+ "id uuid,"
+ "columnfamily_name text,"
@@ -310,8 +187,9 @@ public final class SystemKeyspace
+ "keyspace_name text,"
+ "PRIMARY KEY ((id)))");
- private static final CFMetaData CompactionHistoryTable =
- compile(COMPACTION_HISTORY_TABLE, "week-long compaction history",
+ private static final CFMetaData CompactionHistory =
+ compile(COMPACTION_HISTORY,
+ "week-long compaction history",
"CREATE TABLE %s ("
+ "id uuid,"
+ "bytes_in bigint,"
@@ -321,10 +199,11 @@ public final class SystemKeyspace
+ "keyspace_name text,"
+ "rows_merged map<int, bigint>,"
+ "PRIMARY KEY ((id)))")
- .defaultTimeToLive(WEEK);
+ .defaultTimeToLive((int) TimeUnit.DAYS.toSeconds(7));
- private static final CFMetaData SSTableActivityTable =
- compile(SSTABLE_ACTIVITY_TABLE, "historic sstable read rates",
+ private static final CFMetaData SSTableActivity =
+ compile(SSTABLE_ACTIVITY,
+ "historic sstable read rates",
"CREATE TABLE %s ("
+ "keyspace_name text,"
+ "columnfamily_name text,"
@@ -333,37 +212,29 @@ public final class SystemKeyspace
+ "rate_15m double,"
+ "PRIMARY KEY ((keyspace_name, columnfamily_name, generation)))");
- private static CFMetaData compile(String table, String comment, String cql)
+ private static CFMetaData compile(String name, String description, String schema)
{
- return CFMetaData.compile(String.format(cql, table), NAME).comment(comment);
+ return CFMetaData.compile(String.format(schema, name), NAME).comment(description);
}
public static KSMetaData definition()
{
- List<CFMetaData> tables =
- Arrays.asList(SchemaKeyspacesTable,
- SchemaColumnFamiliesTable,
- SchemaColumnsTable,
- SchemaTriggersTable,
- SchemaUserTypesTable,
- SchemaFunctionsTable,
- SchemaAggregatesTable,
- BuiltIndexesTable,
- HintsTable,
- BatchlogTable,
- PaxosTable,
- LocalTable,
- PeersTable,
- PeerEventsTable,
- RangeXfersTable,
- CompactionLogTable,
- CompactionHistoryTable,
- SSTableActivityTable);
+ Iterable<CFMetaData> tables =
+ Iterables.concat(LegacySchemaTables.All,
+ Arrays.asList(BuiltIndexes,
+ Hints,
+ Batchlog,
+ Paxos,
+ Local,
+ Peers,
+ PeerEvents,
+ RangeXfers,
+ CompactionsInProgress,
+ CompactionHistory,
+ SSTableActivity));
return new KSMetaData(NAME, LocalStrategy.class, Collections.<String, String>emptyMap(), true, tables);
}
- private static final String LOCAL_KEY = "local";
-
private static volatile Map<UUID, Pair<ReplayPosition, Long>> truncationRecords;
public enum BootstrapState
@@ -381,24 +252,15 @@ public final class SystemKeyspace
public static void finishStartup()
{
setupVersion();
-
- // add entries to system schema columnfamilies for the hardcoded system definitions
- KSMetaData ksmd = Schema.instance.getKSMetaData(NAME);
-
- // delete old, possibly obsolete entries in schema tables
- for (String table : ALL_SCHEMA_TABLES)
- executeOnceInternal(String.format("DELETE FROM system.%s WHERE keyspace_name = ?", table), ksmd.name);
-
- // (+1 to timestamp to make sure we don't get shadowed by the tombstones we just added)
- ksmd.toSchema(FBUtilities.timestampMicros() + 1).apply();
+ LegacySchemaTables.saveSystemKeyspaceSchema();
}
private static void setupVersion()
{
String req = "INSERT INTO system.%s (key, release_version, cql_version, thrift_version, native_protocol_version, data_center, rack, partitioner) VALUES (?, ?, ?, ?, ?, ?, ?, ?)";
IEndpointSnitch snitch = DatabaseDescriptor.getEndpointSnitch();
- executeOnceInternal(String.format(req, LOCAL_TABLE),
- LOCAL_KEY,
+ executeOnceInternal(String.format(req, LOCAL),
+ LOCAL,
FBUtilities.getReleaseVersionString(),
QueryProcessor.CQL_VERSION.toString(),
cassandraConstants.VERSION,
@@ -429,8 +291,8 @@ public final class SystemKeyspace
}
});
String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, inputs) VALUES (?, ?, ?, ?)";
- executeInternal(String.format(req, COMPACTION_LOG_TABLE), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
- forceBlockingFlush(COMPACTION_LOG_TABLE);
+ executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS), compactionId, cfs.keyspace.getName(), cfs.name, Sets.newHashSet(generations));
+ forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
return compactionId;
}
@@ -443,8 +305,8 @@ public final class SystemKeyspace
{
assert taskId != null;
- executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTION_LOG_TABLE), taskId);
- forceBlockingFlush(COMPACTION_LOG_TABLE);
+ executeInternal(String.format("DELETE FROM system.%s WHERE id = ?", COMPACTIONS_IN_PROGRESS), taskId);
+ forceBlockingFlush(COMPACTIONS_IN_PROGRESS);
}
/**
@@ -454,7 +316,7 @@ public final class SystemKeyspace
public static Map<Pair<String, String>, Map<Integer, UUID>> getUnfinishedCompactions()
{
String req = "SELECT * FROM system.%s";
- UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTION_LOG_TABLE));
+ UntypedResultSet resultSet = executeInternal(String.format(req, COMPACTIONS_IN_PROGRESS));
Map<Pair<String, String>, Map<Integer, UUID>> unfinishedCompactions = new HashMap<>();
for (UntypedResultSet.Row row : resultSet)
@@ -479,7 +341,7 @@ public final class SystemKeyspace
public static void discardCompactionsInProgress()
{
- ColumnFamilyStore compactionLog = Keyspace.open(NAME).getColumnFamilyStore(COMPACTION_LOG_TABLE);
+ ColumnFamilyStore compactionLog = Keyspace.open(NAME).getColumnFamilyStore(COMPACTIONS_IN_PROGRESS);
compactionLog.truncateBlocking();
}
@@ -491,24 +353,24 @@ public final class SystemKeyspace
Map<Integer, Long> rowsMerged)
{
// don't write anything when the history table itself is compacted, since that would in turn cause new compactions
- if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY_TABLE))
+ if (ksname.equals("system") && cfname.equals(COMPACTION_HISTORY))
return;
String req = "INSERT INTO system.%s (id, keyspace_name, columnfamily_name, compacted_at, bytes_in, bytes_out, rows_merged) VALUES (?, ?, ?, ?, ?, ?, ?)";
- executeInternal(String.format(req, COMPACTION_HISTORY_TABLE), UUIDGen.getTimeUUID(), ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, rowsMerged);
+ executeInternal(String.format(req, COMPACTION_HISTORY), UUIDGen.getTimeUUID(), ksname, cfname, ByteBufferUtil.bytes(compactedAt), bytesIn, bytesOut, rowsMerged);
}
public static TabularData getCompactionHistory() throws OpenDataException
{
- UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY_TABLE));
+ UntypedResultSet queryResultSet = executeInternal(String.format("SELECT * from system.%s", COMPACTION_HISTORY));
return CompactionHistoryTabularData.from(queryResultSet);
}
public static synchronized void saveTruncationRecord(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
{
String req = "UPDATE system.%s SET truncated_at = truncated_at + ? WHERE key = '%s'";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), truncationAsMapEntry(cfs, truncatedAt, position));
+ executeInternal(String.format(req, LOCAL, LOCAL), truncationAsMapEntry(cfs, truncatedAt, position));
truncationRecords = null;
- forceBlockingFlush(LOCAL_TABLE);
+ forceBlockingFlush(LOCAL);
}
/**
@@ -517,9 +379,9 @@ public final class SystemKeyspace
public static synchronized void removeTruncationRecord(UUID cfId)
{
String req = "DELETE truncated_at[?] from system.%s WHERE key = '%s'";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), cfId);
+ executeInternal(String.format(req, LOCAL, LOCAL), cfId);
truncationRecords = null;
- forceBlockingFlush(LOCAL_TABLE);
+ forceBlockingFlush(LOCAL);
}
private static Map<UUID, ByteBuffer> truncationAsMapEntry(ColumnFamilyStore cfs, long truncatedAt, ReplayPosition position)
@@ -558,7 +420,7 @@ public final class SystemKeyspace
private static Map<UUID, Pair<ReplayPosition, Long>> readTruncationRecords()
{
- UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL_TABLE, LOCAL_KEY));
+ UntypedResultSet rows = executeInternal(String.format("SELECT truncated_at FROM system.%s WHERE key = '%s'", LOCAL, LOCAL));
Map<UUID, Pair<ReplayPosition, Long>> records = new HashMap<>();
@@ -597,14 +459,14 @@ public final class SystemKeyspace
}
String req = "INSERT INTO system.%s (peer, tokens) VALUES (?, ?)";
- executeInternal(String.format(req, PEERS_TABLE), ep, tokensAsSet(tokens));
+ executeInternal(String.format(req, PEERS), ep, tokensAsSet(tokens));
}
public static synchronized void updatePreferredIP(InetAddress ep, InetAddress preferred_ip)
{
String req = "INSERT INTO system.%s (peer, preferred_ip) VALUES (?, ?)";
- executeInternal(String.format(req, PEERS_TABLE), ep, preferred_ip);
- forceBlockingFlush(PEERS_TABLE);
+ executeInternal(String.format(req, PEERS), ep, preferred_ip);
+ forceBlockingFlush(PEERS);
}
public static synchronized void updatePeerInfo(InetAddress ep, String columnName, Object value)
@@ -613,20 +475,20 @@ public final class SystemKeyspace
return;
String req = "INSERT INTO system.%s (peer, %s) VALUES (?, ?)";
- executeInternal(String.format(req, PEERS_TABLE, columnName), ep, value);
+ executeInternal(String.format(req, PEERS, columnName), ep, value);
}
public static synchronized void updateHintsDropped(InetAddress ep, UUID timePeriod, int value)
{
// with 30 day TTL
String req = "UPDATE system.%s USING TTL 2592000 SET hints_dropped[ ? ] = ? WHERE peer = ?";
- executeInternal(String.format(req, PEER_EVENTS_TABLE), timePeriod, value, ep);
+ executeInternal(String.format(req, PEER_EVENTS), timePeriod, value, ep);
}
public static synchronized void updateSchemaVersion(UUID version)
{
String req = "INSERT INTO system.%s (key, schema_version) VALUES ('%s', ?)";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), version);
+ executeInternal(String.format(req, LOCAL, LOCAL), version);
}
private static Set<String> tokensAsSet(Collection<Token> tokens)
@@ -653,7 +515,7 @@ public final class SystemKeyspace
public static synchronized void removeEndpoint(InetAddress ep)
{
String req = "DELETE FROM system.%s WHERE peer = ?";
- executeInternal(String.format(req, PEERS_TABLE), ep);
+ executeInternal(String.format(req, PEERS), ep);
}
/**
@@ -663,8 +525,8 @@ public final class SystemKeyspace
{
assert !tokens.isEmpty() : "removeEndpoint should be used instead";
String req = "INSERT INTO system.%s (key, tokens) VALUES ('%s', ?)";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), tokensAsSet(tokens));
- forceBlockingFlush(LOCAL_TABLE);
+ executeInternal(String.format(req, LOCAL, LOCAL), tokensAsSet(tokens));
+ forceBlockingFlush(LOCAL);
}
/**
@@ -696,7 +558,7 @@ public final class SystemKeyspace
public static SetMultimap<InetAddress, Token> loadTokens()
{
SetMultimap<InetAddress, Token> tokenMap = HashMultimap.create();
- for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS_TABLE))
+ for (UntypedResultSet.Row row : executeInternal("SELECT peer, tokens FROM system." + PEERS))
{
InetAddress peer = row.getInetAddress("peer");
if (row.has("tokens"))
@@ -713,7 +575,7 @@ public final class SystemKeyspace
public static Map<InetAddress, UUID> loadHostIds()
{
Map<InetAddress, UUID> hostIdMap = new HashMap<>();
- for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS_TABLE))
+ for (UntypedResultSet.Row row : executeInternal("SELECT peer, host_id FROM system." + PEERS))
{
InetAddress peer = row.getInetAddress("peer");
if (row.has("host_id"))
@@ -733,7 +595,7 @@ public final class SystemKeyspace
public static InetAddress getPreferredIP(InetAddress ep)
{
String req = "SELECT preferred_ip FROM system.%s WHERE peer=?";
- UntypedResultSet result = executeInternal(String.format(req, PEERS_TABLE), ep);
+ UntypedResultSet result = executeInternal(String.format(req, PEERS), ep);
if (!result.isEmpty() && result.one().has("preferred_ip"))
return result.one().getInetAddress("preferred_ip");
return ep;
@@ -745,7 +607,7 @@ public final class SystemKeyspace
public static Map<InetAddress, Map<String,String>> loadDcRackInfo()
{
Map<InetAddress, Map<String, String>> result = new HashMap<>();
- for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS_TABLE))
+ for (UntypedResultSet.Row row : executeInternal("SELECT peer, data_center, rack from system." + PEERS))
{
InetAddress peer = row.getInetAddress("peer");
if (row.has("data_center") && row.has("rack"))
@@ -780,10 +642,10 @@ public final class SystemKeyspace
ex.initCause(err);
throw ex;
}
- ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL_TABLE);
+ ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(LOCAL);
String req = "SELECT cluster_name FROM system.%s WHERE key='%s'";
- UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+ UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
if (result.isEmpty() || !result.one().has("cluster_name"))
{
@@ -793,7 +655,7 @@ public final class SystemKeyspace
// no system files. this is a new node.
req = "INSERT INTO system.%s (key, cluster_name) VALUES ('%s', ?)";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), DatabaseDescriptor.getClusterName());
+ executeInternal(String.format(req, LOCAL, LOCAL), DatabaseDescriptor.getClusterName());
return;
}
@@ -805,7 +667,7 @@ public final class SystemKeyspace
public static Collection<Token> getSavedTokens()
{
String req = "SELECT tokens FROM system.%s WHERE key='%s'";
- UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+ UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
return result.isEmpty() || !result.one().has("tokens")
? Collections.<Token>emptyList()
: deserializeTokens(result.one().getSet("tokens", UTF8Type.instance));
@@ -814,7 +676,7 @@ public final class SystemKeyspace
public static int incrementAndGetGeneration()
{
String req = "SELECT gossip_generation FROM system.%s WHERE key='%s'";
- UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+ UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
int generation;
if (result.isEmpty() || !result.one().has("gossip_generation"))
@@ -842,8 +704,8 @@ public final class SystemKeyspace
}
req = "INSERT INTO system.%s (key, gossip_generation) VALUES ('%s', ?)";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), generation);
- forceBlockingFlush(LOCAL_TABLE);
+ executeInternal(String.format(req, LOCAL, LOCAL), generation);
+ forceBlockingFlush(LOCAL);
return generation;
}
@@ -851,7 +713,7 @@ public final class SystemKeyspace
public static BootstrapState getBootstrapState()
{
String req = "SELECT bootstrapped FROM system.%s WHERE key='%s'";
- UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+ UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
if (result.isEmpty() || !result.one().has("bootstrapped"))
return BootstrapState.NEEDS_BOOTSTRAP;
@@ -872,15 +734,15 @@ public final class SystemKeyspace
public static void setBootstrapState(BootstrapState state)
{
String req = "INSERT INTO system.%s (key, bootstrapped) VALUES ('%s', ?)";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), state.name());
- forceBlockingFlush(LOCAL_TABLE);
+ executeInternal(String.format(req, LOCAL, LOCAL), state.name());
+ forceBlockingFlush(LOCAL);
}
public static boolean isIndexBuilt(String keyspaceName, String indexName)
{
- ColumnFamilyStore cfs = Keyspace.open(NAME).getColumnFamilyStore(BUILT_INDEXES_TABLE);
+ ColumnFamilyStore cfs = Keyspace.open(NAME).getColumnFamilyStore(BUILT_INDEXES);
QueryFilter filter = QueryFilter.getNamesFilter(decorate(ByteBufferUtil.bytes(keyspaceName)),
- BUILT_INDEXES_TABLE,
+ BUILT_INDEXES,
FBUtilities.singleton(cfs.getComparator().makeCellName(indexName), cfs.getComparator()),
System.currentTimeMillis());
return ColumnFamilyStore.removeDeleted(cfs.getColumnFamily(filter), Integer.MAX_VALUE) != null;
@@ -888,7 +750,7 @@ public final class SystemKeyspace
public static void setIndexBuilt(String keyspaceName, String indexName)
{
- ColumnFamily cf = ArrayBackedSortedColumns.factory.create(NAME, BUILT_INDEXES_TABLE);
+ ColumnFamily cf = ArrayBackedSortedColumns.factory.create(NAME, BUILT_INDEXES);
cf.addColumn(new BufferCell(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
new Mutation(NAME, ByteBufferUtil.bytes(keyspaceName), cf).apply();
}
@@ -896,7 +758,7 @@ public final class SystemKeyspace
public static void setIndexRemoved(String keyspaceName, String indexName)
{
Mutation mutation = new Mutation(NAME, ByteBufferUtil.bytes(keyspaceName));
- mutation.delete(BUILT_INDEXES_TABLE, BuiltIndexesTable.comparator.makeCellName(indexName), FBUtilities.timestampMicros());
+ mutation.delete(BUILT_INDEXES, BuiltIndexes.comparator.makeCellName(indexName), FBUtilities.timestampMicros());
mutation.apply();
}
@@ -907,7 +769,7 @@ public final class SystemKeyspace
public static UUID getLocalHostId()
{
String req = "SELECT host_id FROM system.%s WHERE key='%s'";
- UntypedResultSet result = executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY));
+ UntypedResultSet result = executeInternal(String.format(req, LOCAL, LOCAL));
// Look up the Host UUID (return it if found)
if (!result.isEmpty() && result.one().has("host_id"))
@@ -925,144 +787,14 @@ public final class SystemKeyspace
public static UUID setLocalHostId(UUID hostId)
{
String req = "INSERT INTO system.%s (key, host_id) VALUES ('%s', ?)";
- executeInternal(String.format(req, LOCAL_TABLE, LOCAL_KEY), hostId);
+ executeInternal(String.format(req, LOCAL, LOCAL), hostId);
return hostId;
}
- /**
- * @param cfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
- * @return CFS responsible to hold low-level serialized schema
- */
- public static ColumnFamilyStore schemaCFS(String cfName)
- {
- return Keyspace.open(NAME).getColumnFamilyStore(cfName);
- }
-
- public static List<Row> serializedSchema()
- {
- List<Row> schema = new ArrayList<>();
-
- for (String cf : ALL_SCHEMA_TABLES)
- schema.addAll(serializedSchema(cf));
-
- return schema;
- }
-
- /**
- * @param schemaCfName The name of the ColumnFamily responsible for part of the schema (keyspace, ColumnFamily, columns)
- * @return low-level schema representation (each row represents individual Keyspace or ColumnFamily)
- */
- public static List<Row> serializedSchema(String schemaCfName)
- {
- Token minToken = StorageService.getPartitioner().getMinimumToken();
-
- return schemaCFS(schemaCfName).getRangeSlice(new Range<RowPosition>(minToken.minKeyBound(), minToken.maxKeyBound()),
- null,
- new IdentityQueryFilter(),
- Integer.MAX_VALUE,
- System.currentTimeMillis());
- }
-
- public static Collection<Mutation> serializeSchema()
- {
- Map<DecoratedKey, Mutation> mutationMap = new HashMap<>();
-
- for (String cf : ALL_SCHEMA_TABLES)
- serializeSchema(mutationMap, cf);
-
- return mutationMap.values();
- }
-
- private static void serializeSchema(Map<DecoratedKey, Mutation> mutationMap, String schemaCfName)
- {
- for (Row schemaRow : serializedSchema(schemaCfName))
- {
- if (Schema.ignoredSchemaRow(schemaRow))
- continue;
-
- Mutation mutation = mutationMap.get(schemaRow.key);
- if (mutation == null)
- {
- mutation = new Mutation(NAME, schemaRow.key.getKey());
- mutationMap.put(schemaRow.key, mutation);
- }
-
- mutation.add(schemaRow.cf);
- }
- }
-
- public static Map<DecoratedKey, ColumnFamily> getSchema(String cfName)
- {
- Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
-
- for (Row schemaEntity : SystemKeyspace.serializedSchema(cfName))
- schema.put(schemaEntity.key, schemaEntity.cf);
-
- return schema;
- }
-
- public static Map<DecoratedKey, ColumnFamily> getSchema(String schemaCfName, Set<String> keyspaces)
- {
- Map<DecoratedKey, ColumnFamily> schema = new HashMap<>();
-
- for (String keyspace : keyspaces)
- {
- Row schemaEntity = readSchemaRow(schemaCfName, keyspace);
- if (schemaEntity.cf != null)
- schema.put(schemaEntity.key, schemaEntity.cf);
- }
-
- return schema;
- }
-
- public static ByteBuffer getSchemaKSKey(String ksName)
- {
- return AsciiType.instance.fromString(ksName);
- }
-
- /**
- * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace.
- *
- * @param schemaCfName the schema table to get the data from (schema_keyspaces, schema_columnfamilies, schema_columns or schema_triggers)
- * @param ksName the keyspace of the tables we are interested in
- * @return a Row containing the schema data of a particular type for the keyspace
- */
- public static Row readSchemaRow(String schemaCfName, String ksName)
- {
- DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
-
- ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
- ColumnFamily result = schemaCFS.getColumnFamily(QueryFilter.getIdentityFilter(key, schemaCfName, System.currentTimeMillis()));
-
- return new Row(key, result);
- }
-
- /**
- * Fetches a subset of schema (table data, columns metadata or triggers) for the keyspace+table pair.
- *
- * @param schemaCfName the schema table to get the data from (schema_columnfamilies, schema_columns or schema_triggers)
- * @param ksName the keyspace of the table we are interested in
- * @param cfName the table we are interested in
- * @return a Row containing the schema data of a particular type for the table
- */
- public static Row readSchemaRow(String schemaCfName, String ksName, String cfName)
- {
- DecoratedKey key = StorageService.getPartitioner().decorateKey(getSchemaKSKey(ksName));
- ColumnFamilyStore schemaCFS = SystemKeyspace.schemaCFS(schemaCfName);
- Composite prefix = schemaCFS.getComparator().make(cfName);
- ColumnFamily cf = schemaCFS.getColumnFamily(key,
- prefix,
- prefix.end(),
- false,
- Integer.MAX_VALUE,
- System.currentTimeMillis());
- return new Row(key, cf);
- }
-
public static PaxosState loadPaxosState(ByteBuffer key, CFMetaData metadata)
{
String req = "SELECT * FROM system.%s WHERE row_key = ? AND cf_id = ?";
- UntypedResultSet results = executeInternal(String.format(req, PAXOS_TABLE), key, metadata.cfId);
+ UntypedResultSet results = executeInternal(String.format(req, PAXOS), key, metadata.cfId);
if (results.isEmpty())
return new PaxosState(key, metadata);
UntypedResultSet.Row row = results.one();
@@ -1083,7 +815,7 @@ public final class SystemKeyspace
public static void savePaxosPromise(Commit promise)
{
String req = "UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET in_progress_ballot = ? WHERE row_key = ? AND cf_id = ?";
- executeInternal(String.format(req, PAXOS_TABLE),
+ executeInternal(String.format(req, PAXOS),
UUIDGen.microsTimestamp(promise.ballot),
paxosTtl(promise.update.metadata),
promise.ballot,
@@ -1093,7 +825,7 @@ public final class SystemKeyspace
public static void savePaxosProposal(Commit proposal)
{
- executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ? WHERE row_key = ? AND cf_id = ?", PAXOS_TABLE),
+ executeInternal(String.format("UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = ?, proposal = ? WHERE row_key = ? AND cf_id = ?", PAXOS),
UUIDGen.microsTimestamp(proposal.ballot),
paxosTtl(proposal.update.metadata),
proposal.ballot,
@@ -1113,7 +845,7 @@ public final class SystemKeyspace
// We always erase the last proposal (with the commit timestamp to no erase more recent proposal in case the commit is old)
// even though that's really just an optimization since SP.beginAndRepairPaxos will exclude accepted proposal older than the mrc.
String cql = "UPDATE system.%s USING TIMESTAMP ? AND TTL ? SET proposal_ballot = null, proposal = null, most_recent_commit_at = ?, most_recent_commit = ? WHERE row_key = ? AND cf_id = ?";
- executeInternal(String.format(cql, PAXOS_TABLE),
+ executeInternal(String.format(cql, PAXOS),
UUIDGen.microsTimestamp(commit.ballot),
paxosTtl(commit.update.metadata),
commit.ballot,
@@ -1132,7 +864,7 @@ public final class SystemKeyspace
public static RestorableMeter getSSTableReadMeter(String keyspace, String table, int generation)
{
String cql = "SELECT * FROM system.%s WHERE keyspace_name=? and columnfamily_name=? and generation=?";
- UntypedResultSet results = executeInternal(String.format(cql, SSTABLE_ACTIVITY_TABLE), keyspace, table, generation);
+ UntypedResultSet results = executeInternal(String.format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
if (results.isEmpty())
return new RestorableMeter();
@@ -1150,7 +882,7 @@ public final class SystemKeyspace
{
// Store values with a one-day TTL to handle corner cases where cleanup might not occur
String cql = "INSERT INTO system.%s (keyspace_name, columnfamily_name, generation, rate_15m, rate_120m) VALUES (?, ?, ?, ?, ?) USING TTL 864000";
- executeInternal(String.format(cql, SSTABLE_ACTIVITY_TABLE),
+ executeInternal(String.format(cql, SSTABLE_ACTIVITY),
keyspace,
table,
generation,
@@ -1164,6 +896,6 @@ public final class SystemKeyspace
public static void clearSSTableReadMeter(String keyspace, String table, int generation)
{
String cql = "DELETE FROM system.%s WHERE keyspace_name=? AND columnfamily_name=? and generation=?";
- executeInternal(String.format(cql, SSTABLE_ACTIVITY_TABLE), keyspace, table, generation);
+ executeInternal(String.format(cql, SSTABLE_ACTIVITY), keyspace, table, generation);
}
}
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index 0b52904..8be9a18 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,17 +24,19 @@ import java.nio.ByteBuffer;
import java.util.*;
import com.google.common.collect.*;
-import org.apache.cassandra.db.BufferCell;
-import org.apache.cassandra.db.Cell;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.BufferCell;
+import org.apache.cassandra.db.Cell;
import org.apache.cassandra.db.composites.CellNames;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.CompositeType;
import org.apache.cassandra.db.marshal.TypeParser;
import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.thrift.*;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
@@ -228,17 +230,15 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
{
partitioner = FBUtilities.newPartitioner(client.describe_partitioner());
// get CF meta data
- String query = "SELECT comparator," +
- " subcomparator," +
- " type " +
- "FROM system.schema_columnfamilies " +
- "WHERE keyspace_name = '%s' " +
- " AND columnfamily_name = '%s' ";
-
- CqlResult result = client.execute_cql3_query(
- ByteBufferUtil.bytes(String.format(query, keyspace, cfName)),
- Compression.NONE,
- ConsistencyLevel.ONE);
+ String query = String.format("SELECT comparator, subcomparator, type " +
+ "FROM %s.%s " +
+ "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+ SystemKeyspace.NAME,
+ LegacySchemaTables.COLUMNFAMILIES,
+ keyspace,
+ cfName);
+
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
Iterator<CqlRow> iteraRow = result.rows.iterator();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
index 21e30e2..ffaaea9 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java
@@ -22,17 +22,7 @@ import java.math.BigDecimal;
import java.math.BigInteger;
import java.net.InetAddress;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
import com.google.common.base.Function;
import com.google.common.base.Joiner;
@@ -40,10 +30,7 @@ import com.google.common.base.Splitter;
import com.google.common.collect.AbstractIterator;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
-
import org.apache.commons.lang3.StringUtils;
-
-import org.apache.cassandra.hadoop.HadoopCompat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -55,6 +42,8 @@ import com.datastax.driver.core.Row;
import com.datastax.driver.core.Session;
import com.datastax.driver.core.TupleValue;
import com.datastax.driver.core.UDTValue;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.hadoop.ColumnFamilySplit;
@@ -601,8 +590,15 @@ public class CqlRecordReader extends RecordReader<Long, Row>
private void fetchKeys()
{
- String query = "SELECT column_name, component_index, type FROM system.schema_columns WHERE keyspace_name='" +
- keyspace + "' and columnfamily_name='" + cfName + "'";
+ String query = String.format("SELECT column_name, component_index, type " +
+ "FROM %s.%s " +
+ "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+ SystemKeyspace.NAME,
+ LegacySchemaTables.COLUMNS,
+ keyspace,
+ cfName);
+
+ // get CF meta data
List<Row> rows = session.execute(query).all();
if (rows.isEmpty())
{
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
index 311359a..0956ba5 100644
--- a/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordWriter.java
@@ -25,6 +25,9 @@ import java.util.concurrent.ConcurrentHashMap;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.CompositeType;
import org.apache.cassandra.db.marshal.LongType;
@@ -297,10 +300,6 @@ class CqlRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteB
{
result = client.prepare_cql3_query(ByteBufferUtil.bytes(cql), Compression.NONE);
}
- catch (InvalidRequestException e)
- {
- throw new RuntimeException("failed to prepare cql query " + cql, e);
- }
catch (TException e)
{
throw new RuntimeException("failed to prepare cql query " + cql, e);
@@ -331,18 +330,20 @@ class CqlRecordWriter extends AbstractColumnFamilyRecordWriter<Map<String, ByteB
return partitionKey;
}
+ // FIXME
/** retrieve the key validator from system.schema_columnfamilies table */
private void retrievePartitionKeyValidator(Cassandra.Client client) throws Exception
{
String keyspace = ConfigHelper.getOutputKeyspace(conf);
String cfName = ConfigHelper.getOutputColumnFamily(conf);
- String query = "SELECT key_validator," +
- " key_aliases," +
- " column_aliases " +
- "FROM system.schema_columnfamilies " +
- "WHERE keyspace_name='%s' and columnfamily_name='%s'";
- String formatted = String.format(query, keyspace, cfName);
- CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(formatted), Compression.NONE, ConsistencyLevel.ONE);
+ String query = String.format("SELECT key_validator, key_aliases, column_aliases " +
+ "FROM %s.%s " +
+ "WHERE keyspace_name = '%s' and columnfamily_name = '%s'",
+ SystemKeyspace.NAME,
+ LegacySchemaTables.COLUMNFAMILIES,
+ keyspace,
+ cfName);
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
Column rawKeyValidator = result.rows.get(0).columns.get(0);
String validator = ByteBufferUtil.string(ByteBuffer.wrap(rawKeyValidator.getValue()));
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index dc37252..04d207f 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -26,6 +26,8 @@ import java.nio.charset.CharacterCodingException;
import java.util.*;
import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.exceptions.ConfigurationException;
import org.apache.cassandra.exceptions.SyntaxException;
import org.apache.cassandra.auth.IAuthenticator;
@@ -585,20 +587,15 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
IOException
{
// get CF meta data
- String query = "SELECT type," +
- " comparator," +
- " subcomparator," +
- " default_validator," +
- " key_validator," +
- " key_aliases " +
- "FROM system.schema_columnfamilies " +
- "WHERE keyspace_name = '%s' " +
- " AND columnfamily_name = '%s' ";
-
- CqlResult result = client.execute_cql3_query(
- ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
- Compression.NONE,
- ConsistencyLevel.ONE);
+ String query = String.format("SELECT type, comparator, subcomparator, default_validator, key_validator, key_aliases " +
+ "FROM %s.%s " +
+ "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+ SystemKeyspace.NAME,
+ LegacySchemaTables.COLUMNFAMILIES,
+ keyspace,
+ column_family);
+
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
if (result == null || result.rows == null || result.rows.isEmpty())
return null;
@@ -657,18 +654,15 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
ConfigurationException,
NotFoundException
{
- String query = "SELECT column_name, " +
- " validator, " +
- " index_type, " +
- " type " +
- "FROM system.schema_columns " +
- "WHERE keyspace_name = '%s' " +
- " AND columnfamily_name = '%s'";
-
- CqlResult result = client.execute_cql3_query(
- ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
- Compression.NONE,
- ConsistencyLevel.ONE);
+ String query = String.format("SELECT column_name, validator, index_type, type " +
+ "FROM %s.%s " +
+ "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+ SystemKeyspace.NAME,
+ LegacySchemaTables.COLUMNS,
+ keyspace,
+ column_family);
+
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
List<CqlRow> rows = result.rows;
List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index 6cd5c66..fca1d43 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -25,6 +25,8 @@ import java.util.*;
import org.apache.cassandra.config.CFMetaData;
import org.apache.cassandra.config.ColumnDefinition;
import org.apache.cassandra.db.BufferCell;
+import org.apache.cassandra.schema.LegacySchemaTables;
+import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.db.composites.CellNames;
import org.apache.cassandra.db.Cell;
import org.apache.cassandra.db.marshal.*;
@@ -482,21 +484,15 @@ public class CqlStorage extends AbstractCassandraStorage
protected List<ColumnDef> getKeysMeta(Cassandra.Client client)
throws Exception
{
- String query = "SELECT key_aliases, " +
- " column_aliases, " +
- " key_validator, " +
- " comparator, " +
- " keyspace_name, " +
- " value_alias, " +
- " default_validator " +
- "FROM system.schema_columnfamilies " +
- "WHERE keyspace_name = '%s'" +
- " AND columnfamily_name = '%s' ";
-
- CqlResult result = client.execute_cql3_query(
- ByteBufferUtil.bytes(String.format(query, keyspace, column_family)),
- Compression.NONE,
- ConsistencyLevel.ONE);
+ String query = String.format("SELECT key_aliases, column_aliases, key_validator, comparator, keyspace_name, value_alias, default_validator " +
+ "FROM %s.%s " +
+ "WHERE keyspace_name = '%s' AND columnfamily_name = '%s'",
+ SystemKeyspace.NAME,
+ LegacySchemaTables.COLUMNFAMILIES,
+ keyspace,
+ column_family);
+
+ CqlResult result = client.execute_cql3_query(ByteBufferUtil.bytes(query), Compression.NONE, ConsistencyLevel.ONE);
if (result == null || result.rows == null || result.rows.isEmpty())
return null;
http://git-wip-us.apache.org/repos/asf/cassandra/blob/3e9d345f/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 43cd2c0..ec590f3 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -349,16 +349,16 @@ public class CQLSSTableWriter implements Closeable
if (ksm == null)
{
ksm = KSMetaData.newKeyspace(this.schema.ksName,
- AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
- ImmutableMap.of("replication_factor", "1"),
- true,
- Collections.singleton(this.schema));
+ AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
+ ImmutableMap.of("replication_factor", "1"),
+ true,
+ Collections.singleton(this.schema));
Schema.instance.load(ksm);
}
else if (Schema.instance.getCFMetaData(this.schema.ksName, this.schema.cfName) == null)
{
Schema.instance.load(this.schema);
- ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(), Collections.singleton(this.schema)));
+ ksm = ksm.cloneWithTableAdded(this.schema);
Schema.instance.setKeyspaceDefinition(ksm);
Keyspace.open(ksm.name).initCf(this.schema.cfId, this.schema.cfName, false);
}