/** * Given arguments specifying an SSTable, and optionally an output file, export the contents of * the SSTable to JSON. * * @param args command lines arguments * @throws ConfigurationException on configuration failure (wrong params given) */ public static void main(String[] args) throws ConfigurationException { CommandLineParser parser = new PosixParser(); try { cmd = parser.parse(options, args); } catch (ParseException e1) { System.err.println(e1.getMessage()); printUsage(); System.exit(1); } if (cmd.getArgs().length != 1) { System.err.println("You must supply exactly one sstable"); printUsage(); System.exit(1); } String[] keys = cmd.getOptionValues(KEY_OPTION); HashSet<String> excludes = new HashSet<>( Arrays.asList( cmd.getOptionValues(EXCLUDE_KEY_OPTION) == null ? new String[0] : cmd.getOptionValues(EXCLUDE_KEY_OPTION))); String ssTableFileName = new File(cmd.getArgs()[0]).getAbsolutePath(); if (Descriptor.isLegacyFile(new File(ssTableFileName))) { System.err.println("Unsupported legacy sstable"); System.exit(1); } if (!new File(ssTableFileName).exists()) { System.err.println("Cannot find file " + ssTableFileName); System.exit(1); } Descriptor desc = Descriptor.fromFilename(ssTableFileName); try { CFMetaData metadata = metadataFromSSTable(desc); if (cmd.hasOption(ENUMERATE_KEYS_OPTION)) { JsonTransformer.keysToJson( null, iterToStream(new KeyIterator(desc, metadata)), cmd.hasOption(RAW_TIMESTAMPS), metadata, System.out); } else { SSTableReader sstable = SSTableReader.openNoValidation(desc, metadata); IPartitioner partitioner = sstable.getPartitioner(); final ISSTableScanner currentScanner; if ((keys != null) && (keys.length > 0)) { List<AbstractBounds<PartitionPosition>> bounds = Arrays.stream(keys) .filter(key -> !excludes.contains(key)) .map(metadata.getKeyValidator()::fromString) .map(partitioner::decorateKey) .sorted() .map(DecoratedKey::getToken) .map(token -> new Bounds<>(token.minKeyBound(), token.maxKeyBound())) .collect(Collectors.toList()); currentScanner = sstable.getScanner(bounds.iterator()); } else { currentScanner = sstable.getScanner(); } Stream<UnfilteredRowIterator> partitions = iterToStream(currentScanner) .filter( i -> excludes.isEmpty() || !excludes.contains( metadata.getKeyValidator().getString(i.partitionKey().getKey()))); if (cmd.hasOption(DEBUG_OUTPUT_OPTION)) { AtomicLong position = new AtomicLong(); partitions.forEach( partition -> { position.set(currentScanner.getCurrentPosition()); if (!partition.partitionLevelDeletion().isLive()) { System.out.println( "[" + metadata.getKeyValidator().getString(partition.partitionKey().getKey()) + "]@" + position.get() + " " + partition.partitionLevelDeletion()); } if (!partition.staticRow().isEmpty()) { System.out.println( "[" + metadata.getKeyValidator().getString(partition.partitionKey().getKey()) + "]@" + position.get() + " " + partition.staticRow().toString(metadata, true)); } partition.forEachRemaining( row -> { System.out.println( "[" + metadata .getKeyValidator() .getString(partition.partitionKey().getKey()) + "]@" + position.get() + " " + row.toString(metadata, false, true)); position.set(currentScanner.getCurrentPosition()); }); }); } else { JsonTransformer.toJson( currentScanner, partitions, cmd.hasOption(RAW_TIMESTAMPS), metadata, System.out); } } } catch (IOException e) { // throwing exception outside main with broken pipe causes windows cmd to hang e.printStackTrace(System.err); } System.exit(0); }
public ColumnNameBuilder getKeyNameBuilder() { return hasCompositeKey ? new CompositeType.Builder((CompositeType) cfm.getKeyValidator()) : new NonCompositeBuilder(cfm.getKeyValidator()); }
public CFDefinition(CFMetaData cfm) { this.cfm = cfm; if (cfm.getKeyValidator() instanceof CompositeType) { this.hasCompositeKey = true; CompositeType keyComposite = (CompositeType) cfm.getKeyValidator(); assert keyComposite.types.size() > 1; for (int i = 0; i < keyComposite.types.size(); i++) { ColumnIdentifier id = getKeyId(cfm, i); this.keys.put( id, new Name( cfm.ksName, cfm.cfName, id, Name.Kind.KEY_ALIAS, i, keyComposite.types.get(i))); } } else { this.hasCompositeKey = false; ColumnIdentifier id = getKeyId(cfm, 0); this.keys.put( id, new Name(cfm.ksName, cfm.cfName, id, Name.Kind.KEY_ALIAS, 0, cfm.getKeyValidator())); } if (cfm.comparator instanceof CompositeType) { this.isComposite = true; CompositeType composite = (CompositeType) cfm.comparator; /* * We are a "sparse" composite, i.e. a non-compact one, if either: * - the last type of the composite is a ColumnToCollectionType * - or we have one less alias than of composite types and the last type is UTF8Type. * * Note that this is not perfect: if someone upgrading from thrift "renames" all but * the last column alias, the cf will be considered "sparse" and he will be stuck with * that even though that might not be what he wants. But the simple workaround is * for that user to rename all the aliases at the same time in the first place. */ int last = composite.types.size() - 1; AbstractType<?> lastType = composite.types.get(last); if (lastType instanceof ColumnToCollectionType || (cfm.getColumnAliases().size() == last && lastType instanceof UTF8Type)) { // "sparse" composite this.isCompact = false; this.value = null; assert cfm.getValueAlias() == null; // check for collection type if (lastType instanceof ColumnToCollectionType) { --last; this.hasCollections = true; } else { this.hasCollections = false; } for (int i = 0; i < last; i++) { ColumnIdentifier id = getColumnId(cfm, i); this.columns.put( id, new Name( cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_ALIAS, i, composite.types.get(i))); } for (Map.Entry<ByteBuffer, ColumnDefinition> def : cfm.getColumn_metadata().entrySet()) { ColumnIdentifier id = new ColumnIdentifier(def.getKey(), cfm.getColumnDefinitionComparator(def.getValue())); this.metadata.put( id, new Name( cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_METADATA, def.getValue().getValidator())); } } else { // "dense" composite this.isCompact = true; this.hasCollections = false; for (int i = 0; i < composite.types.size(); i++) { ColumnIdentifier id = getColumnId(cfm, i); this.columns.put( id, new Name( cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_ALIAS, i, composite.types.get(i))); } this.value = createValue(cfm); } } else { this.isComposite = false; this.hasCollections = false; if (!cfm.getColumnAliases().isEmpty() || cfm.getColumn_metadata().isEmpty()) { // dynamic CF this.isCompact = true; ColumnIdentifier id = getColumnId(cfm, 0); Name name = new Name(cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_ALIAS, 0, cfm.comparator); this.columns.put(id, name); this.value = createValue(cfm); } else { // static CF this.isCompact = false; this.value = null; assert cfm.getValueAlias() == null; assert cfm.getColumnAliases() == null || cfm.getColumnAliases().isEmpty(); for (Map.Entry<ByteBuffer, ColumnDefinition> def : cfm.getColumn_metadata().entrySet()) { ColumnIdentifier id = new ColumnIdentifier(def.getKey(), cfm.getColumnDefinitionComparator(def.getValue())); this.metadata.put( id, new Name( cfm.ksName, cfm.cfName, id, Name.Kind.COLUMN_METADATA, def.getValue().getValidator())); } } } assert value == null || metadata.isEmpty(); }
public void announceMigration() throws RequestValidationException { CFMetaData meta = validateColumnFamily(keyspace(), columnFamily()); CFMetaData cfm = meta.clone(); CFDefinition cfDef = meta.getCfDef(); CFDefinition.Name name = columnName == null ? null : cfDef.get(columnName); switch (oType) { case ADD: if (cfDef.isCompact) throw new InvalidRequestException("Cannot add new column to a compact CF"); if (name != null) { switch (name.kind) { case KEY_ALIAS: case COLUMN_ALIAS: throw new InvalidRequestException( String.format( "Invalid column name %s because it conflicts with a PRIMARY KEY part", columnName)); case COLUMN_METADATA: throw new InvalidRequestException( String.format( "Invalid column name %s because it conflicts with an existing column", columnName)); } } AbstractType<?> type = validator.getType(); if (type instanceof CollectionType) { if (!cfDef.isComposite) throw new InvalidRequestException( "Cannot use collection types with non-composite PRIMARY KEY"); if (cfDef.cfm.isSuper()) throw new InvalidRequestException( "Cannot use collection types with Super column family"); Map<ByteBuffer, CollectionType> collections = cfDef.hasCollections ? new HashMap<ByteBuffer, CollectionType>(cfDef.getCollectionType().defined) : new HashMap<ByteBuffer, CollectionType>(); collections.put(columnName.key, (CollectionType) type); ColumnToCollectionType newColType = ColumnToCollectionType.getInstance(collections); List<AbstractType<?>> ctypes = new ArrayList<AbstractType<?>>(((CompositeType) cfm.comparator).types); if (cfDef.hasCollections) ctypes.set(ctypes.size() - 1, newColType); else ctypes.add(newColType); cfm.comparator = CompositeType.getInstance(ctypes); } Integer componentIndex = cfDef.isComposite ? ((CompositeType) meta.comparator).types.size() - (cfDef.hasCollections ? 2 : 1) : null; cfm.addColumnDefinition(ColumnDefinition.regularDef(columnName.key, type, componentIndex)); break; case ALTER: if (name == null) throw new InvalidRequestException( String.format("Column %s was not found in table %s", columnName, columnFamily())); switch (name.kind) { case KEY_ALIAS: AbstractType<?> newType = validator.getType(); if (newType instanceof CounterColumnType) throw new InvalidRequestException( String.format( "counter type is not supported for PRIMARY KEY part %s", columnName)); if (cfDef.hasCompositeKey) { List<AbstractType<?>> newTypes = new ArrayList<AbstractType<?>>(((CompositeType) cfm.getKeyValidator()).types); newTypes.set(name.position, newType); cfm.keyValidator(CompositeType.getInstance(newTypes)); } else { cfm.keyValidator(newType); } break; case COLUMN_ALIAS: assert cfDef.isComposite; List<AbstractType<?>> newTypes = new ArrayList<AbstractType<?>>(((CompositeType) cfm.comparator).types); newTypes.set(name.position, validator.getType()); cfm.comparator = CompositeType.getInstance(newTypes); break; case VALUE_ALIAS: cfm.defaultValidator(validator.getType()); break; case COLUMN_METADATA: ColumnDefinition column = cfm.getColumnDefinition(columnName.key); column.setValidator(validator.getType()); break; } break; case DROP: if (cfDef.isCompact) throw new InvalidRequestException("Cannot drop columns from a compact CF"); if (name == null) throw new InvalidRequestException( String.format("Column %s was not found in table %s", columnName, columnFamily())); switch (name.kind) { case KEY_ALIAS: case COLUMN_ALIAS: throw new InvalidRequestException( String.format("Cannot drop PRIMARY KEY part %s", columnName)); case COLUMN_METADATA: ColumnDefinition toDelete = null; for (ColumnDefinition columnDef : cfm.regularColumns()) { if (columnDef.name.equals(columnName.key)) toDelete = columnDef; } assert toDelete != null; cfm.removeColumnDefinition(toDelete); break; } break; case OPTS: if (cfProps == null) throw new InvalidRequestException( String.format("ALTER COLUMNFAMILY WITH invoked, but no parameters found")); cfProps.validate(); cfProps.applyToCFMetadata(cfm); break; case RENAME: for (Map.Entry<ColumnIdentifier, ColumnIdentifier> entry : renames.entrySet()) { ColumnIdentifier from = entry.getKey(); ColumnIdentifier to = entry.getValue(); cfm.renameColumn(from.key, from.toString(), to.key, to.toString()); } break; } MigrationManager.announceColumnFamilyUpdate(cfm, false); }