public KsDef toThrift() { List<CfDef> cfDefs = new ArrayList<CfDef>(); for (CFMetaData cfm : cfMetaData().values()) cfDefs.add(cfm.toThrift()); KsDef ksdef = new KsDef(name, strategyClass.getName(), cfDefs); ksdef.setStrategy_options(strategyOptions); ksdef.setDurable_writes(durableWrites); return ksdef; }
public RowMutation toSchema(long timestamp) { RowMutation rm = new RowMutation(Table.SYSTEM_TABLE, SystemTable.getSchemaKSKey(name)); ColumnFamily cf = rm.addOrGet(SystemTable.SCHEMA_KEYSPACES_CF); cf.addColumn(Column.create(name, timestamp, "name")); cf.addColumn(Column.create(durableWrites, timestamp, "durable_writes")); cf.addColumn(Column.create(strategyClass.getName(), timestamp, "strategy_class")); cf.addColumn(Column.create(json(strategyOptions), timestamp, "strategy_options")); for (CFMetaData cfm : cfMetaData.values()) cfm.toSchema(rm, timestamp); return rm; }
public KSMetaData validate() throws ConfigurationException { if (!CFMetaData.isNameValid(name)) throw new ConfigurationException( String.format( "Invalid keyspace name: shouldn't be empty nor more than %s characters long (got \"%s\")", Schema.NAME_LENGTH, name)); // Attempt to instantiate the ARS, which will throw a ConfigException if the strategy_options // aren't fully formed TokenMetadata tmd = StorageService.instance.getTokenMetadata(); IEndpointSnitch eps = DatabaseDescriptor.getEndpointSnitch(); AbstractReplicationStrategy.createReplicationStrategy( name, strategyClass, tmd, eps, strategyOptions); for (CFMetaData cfm : cfMetaData.values()) cfm.validate(); return this; }
/** * Deserialize ColumnFamilies from low-level schema representation, all of them belong to the same * keyspace * * @param row * @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(); Map<String, CFMetaData> cfms = new HashMap<String, CFMetaData>(); UntypedResultSet results = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", row); for (UntypedResultSet.Row result : results) { CFMetaData cfm = CFMetaData.fromSchema(result); cfms.put(cfm.cfName, cfm); } for (CFMetaData cfm : cfms.values()) { Row columnRow = ColumnDefinition.readSchema(cfm.ksName, cfm.cfName); for (ColumnDefinition cd : ColumnDefinition.fromSchema(columnRow, cfm)) cfm.column_metadata.put(cd.name, cd); } return cfms; }