protected ColumnNameBuilder makeIndexColumnNameBuilder(ByteBuffer rowKey, ByteBuffer columnName) {
   int ckCount = baseCfs.metadata.clusteringKeyColumns().size();
   CompositeType baseComparator = (CompositeType) baseCfs.getComparator();
   ByteBuffer[] components = baseComparator.split(columnName);
   CompositeType.Builder builder = getIndexComparator().builder();
   builder.add(rowKey);
   for (int i = 0; i < ckCount; i++) builder.add(components[i]);
   return builder;
 }
 public static CompositeType buildIndexComparator(
     CFMetaData baseMetadata, ColumnDefinition columnDef) {
   int ckCount = baseMetadata.clusteringKeyColumns().size();
   List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(ckCount + 1);
   types.add(SecondaryIndex.keyComparator);
   types.addAll(baseMetadata.comparator.getComponents());
   return CompositeType.getInstance(types);
 }
 /**
  * Returns a {@code String} representation of {@code byteBuffer} validated by {@code type}.
  *
  * @param byteBuffer the {@link ByteBuffer} to be converted to {@code String}.
  * @param type {@link AbstractType} of {@code byteBuffer}.
  * @return a {@code String} representation of {@code byteBuffer} validated by {@code type}.
  */
 public static String toString(ByteBuffer byteBuffer, AbstractType<?> type) {
   if (type instanceof CompositeType) {
     CompositeType composite = (CompositeType) type;
     List<AbstractType<?>> types = composite.types;
     ByteBuffer[] components = composite.split(byteBuffer);
     StringBuilder sb = new StringBuilder();
     for (int i = 0; i < components.length; i++) {
       AbstractType<?> componentType = types.get(i);
       ByteBuffer component = components[i];
       sb.append(componentType.compose(component));
       if (i < types.size() - 1) {
         sb.append(':');
       }
     }
     return sb.toString();
   } else {
     return type.compose(byteBuffer).toString();
   }
 }
  private ByteBuffer getPartitionKey(Map<String, ByteBuffer> keyColumns) {
    ByteBuffer partitionKey;
    if (partitionKeyColumns.size() > 1) {
      ByteBuffer[] keys = new ByteBuffer[partitionKeyColumns.size()];
      for (int i = 0; i < keys.length; i++)
        keys[i] = keyColumns.get(partitionKeyColumns.get(i).getName());

      partitionKey = CompositeType.build(keys);
    } else {
      partitionKey = keyColumns.get(partitionKeyColumns.get(0).getName());
    }
    return partitionKey;
  }
Пример #5
0
  private static CFMetaData compositeCFMD(String ksName, String cfName, AbstractType... types) {
    try {
      return new CFMetaData(
          ksName,
          cfName,
          ColumnFamilyType.Standard,
          CompositeType.getInstance(Arrays.asList(types)),
          null);
    } catch (ConfigurationException e) {

    }
    return null;
  }
Пример #6
0
  private void testDontPurgeAccidentaly(String k, String cfname)
      throws IOException, ExecutionException, InterruptedException {
    // This test catches the regression of CASSANDRA-2786
    Keyspace keyspace = Keyspace.open(KEYSPACE1);
    ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);

    // disable compaction while flushing
    cfs.clearUnsafe();
    cfs.disableAutoCompaction();

    // Add test row
    DecoratedKey key = Util.dk(k);
    RowMutation rm = new RowMutation(KEYSPACE1, key.key);
    rm.add(
        cfname,
        CompositeType.build(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")),
        ByteBufferUtil.EMPTY_BYTE_BUFFER,
        0);
    rm.apply();

    cfs.forceBlockingFlush();

    Collection<SSTableReader> sstablesBefore = cfs.getSSTables();

    QueryFilter filter = QueryFilter.getIdentityFilter(key, cfname, System.currentTimeMillis());
    assert !(cfs.getColumnFamily(filter).getColumnCount() == 0);

    // Remove key
    rm = new RowMutation(KEYSPACE1, key.key);
    rm.delete(cfname, 2);
    rm.apply();

    ColumnFamily cf = cfs.getColumnFamily(filter);
    assert cf == null || cf.getColumnCount() == 0 : "should be empty: " + cf;

    // Sleep one second so that the removal is indeed purgeable even with gcgrace == 0
    Thread.sleep(1000);

    cfs.forceBlockingFlush();

    Collection<SSTableReader> sstablesAfter = cfs.getSSTables();
    Collection<SSTableReader> toCompact = new ArrayList<SSTableReader>();
    for (SSTableReader sstable : sstablesAfter)
      if (!sstablesBefore.contains(sstable)) toCompact.add(sstable);

    Util.compact(cfs, toCompact);

    cf = cfs.getColumnFamily(filter);
    assert cf == null || cf.getColumnCount() == 0 : "should be empty: " + cf;
  }
Пример #7
0
  @Test
  public void testSuperColumnTombstones()
      throws IOException, ExecutionException, InterruptedException {
    Keyspace keyspace = Keyspace.open(KEYSPACE1);
    ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Super1");
    cfs.disableAutoCompaction();

    DecoratedKey key = Util.dk("tskey");
    ByteBuffer scName = ByteBufferUtil.bytes("TestSuperColumn");

    // a subcolumn
    RowMutation rm = new RowMutation(KEYSPACE1, key.key);
    rm.add(
        "Super1",
        CompositeType.build(scName, ByteBufferUtil.bytes(0)),
        ByteBufferUtil.EMPTY_BYTE_BUFFER,
        FBUtilities.timestampMicros());
    rm.apply();
    cfs.forceBlockingFlush();

    // shadow the subcolumn with a supercolumn tombstone
    rm = new RowMutation(KEYSPACE1, key.key);
    rm.deleteRange(
        "Super1",
        SuperColumns.startOf(scName),
        SuperColumns.endOf(scName),
        FBUtilities.timestampMicros());
    rm.apply();
    cfs.forceBlockingFlush();

    CompactionManager.instance.performMaximal(cfs);
    assertEquals(1, cfs.getSSTables().size());

    // check that the shadowed column is gone
    SSTableReader sstable = cfs.getSSTables().iterator().next();
    Range keyRange =
        new Range<RowPosition>(key, sstable.partitioner.getMinimumToken().maxKeyBound());
    SSTableScanner scanner = sstable.getScanner(DataRange.forKeyRange(keyRange));
    OnDiskAtomIterator iter = scanner.next();
    assertEquals(key, iter.getKey());
    assert iter.next() instanceof RangeTombstone;
    assert !iter.hasNext();
  }
    private Statics() {
      StandardCf.addColumn(new Column(bb("aaaa")));
      StandardCf.addColumn(new Column(bb("bbbb"), bb("bbbbb-value")));
      StandardCf.addColumn(new Column(bb("cccc"), bb("ccccc-value"), 1000L));
      StandardCf.addColumn(new DeletedColumn(bb("dddd"), 500, 1000));
      StandardCf.addColumn(new DeletedColumn(bb("eeee"), bb("eeee-value"), 1001));
      StandardCf.addColumn(new ExpiringColumn(bb("ffff"), bb("ffff-value"), 2000, 1000));
      StandardCf.addColumn(new ExpiringColumn(bb("gggg"), bb("gggg-value"), 2001, 1000, 2002));

      SuperCf.addColumn(new Column(CompositeType.build(SC, bb("aaaa"))));
      SuperCf.addColumn(new Column(CompositeType.build(SC, bb("bbbb")), bb("bbbbb-value")));
      SuperCf.addColumn(new Column(CompositeType.build(SC, bb("cccc")), bb("ccccc-value"), 1000L));
      SuperCf.addColumn(new DeletedColumn(CompositeType.build(SC, bb("dddd")), 500, 1000));
      SuperCf.addColumn(
          new DeletedColumn(CompositeType.build(SC, bb("eeee")), bb("eeee-value"), 1001));
      SuperCf.addColumn(
          new ExpiringColumn(CompositeType.build(SC, bb("ffff")), bb("ffff-value"), 2000, 1000));
      SuperCf.addColumn(
          new ExpiringColumn(
              CompositeType.build(SC, bb("gggg")), bb("gggg-value"), 2001, 1000, 2002));
    }
 protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column) {
   CompositeType keyComparator = (CompositeType) baseCfs.metadata.getKeyValidator();
   ByteBuffer[] components = keyComparator.split(rowKey);
   return components[columnDef.componentIndex];
 }
Пример #10
0
  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);
  }
    /** Transform this raw statement into a CreateTableStatement. */
    public ParsedStatement.Prepared prepare() throws RequestValidationException {
      // Column family name
      if (!columnFamily().matches("\\w+"))
        throw new InvalidRequestException(
            String.format(
                "\"%s\" is not a valid column family name (must be alphanumeric character only: [0-9A-Za-z]+)",
                columnFamily()));
      if (columnFamily().length() > Schema.NAME_LENGTH)
        throw new InvalidRequestException(
            String.format(
                "Column family names shouldn't be more than %s characters long (got \"%s\")",
                Schema.NAME_LENGTH, columnFamily()));

      for (Multiset.Entry<ColumnIdentifier> entry : definedNames.entrySet())
        if (entry.getCount() > 1)
          throw new InvalidRequestException(
              String.format("Multiple definition of identifier %s", entry.getElement()));

      properties.validate();
      CreateTableStatement stmt =
          new CreateTableStatement(cfName, properties, ifNotExists, staticColumns);

      Map<ByteBuffer, CollectionType> definedCollections = null;
      for (Map.Entry<ColumnIdentifier, CQL3Type> entry : definitions.entrySet()) {

        ColumnIdentifier id = entry.getKey();
        CQL3Type pt = entry.getValue();
        if (pt.isCollection()) {
          if (definedCollections == null)
            definedCollections = new HashMap<ByteBuffer, CollectionType>();
          definedCollections.put(id.key, (CollectionType) pt.getType());
        }
        stmt.columns.put(id, pt.getType()); // we'll remove what is not a column below
      }

      if (keyAliases.isEmpty())
        throw new InvalidRequestException("No PRIMARY KEY specifed (exactly one required)");
      else if (keyAliases.size() > 1)
        throw new InvalidRequestException("Multiple PRIMARY KEYs specifed (exactly one required)");

      List<ColumnIdentifier> kAliases = keyAliases.get(0);

      List<AbstractType<?>> keyTypes = new ArrayList<AbstractType<?>>(kAliases.size());
      for (ColumnIdentifier alias : kAliases) {
        stmt.keyAliases.add(alias.key);
        AbstractType<?> t = getTypeAndRemove(stmt.columns, alias);
        if (t instanceof CounterColumnType)
          throw new InvalidRequestException(
              String.format("counter type is not supported for PRIMARY KEY part %s", alias));
        if (staticColumns.contains(alias))
          throw new InvalidRequestException(
              String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
        keyTypes.add(t);
      }
      stmt.keyValidator =
          keyTypes.size() == 1 ? keyTypes.get(0) : CompositeType.getInstance(keyTypes);

      // Dense means that no part of the comparator stores a CQL column name. This means
      // COMPACT STORAGE with at least one columnAliases (otherwise it's a thrift "static" CF).
      stmt.isDense = useCompactStorage && !columnAliases.isEmpty();

      // Handle column aliases
      if (columnAliases.isEmpty()) {
        if (useCompactStorage) {
          // There should remain some column definition since it is a non-composite "static" CF
          if (stmt.columns.isEmpty())
            throw new InvalidRequestException(
                "No definition found that is not part of the PRIMARY KEY");

          if (definedCollections != null)
            throw new InvalidRequestException(
                "Collection types are not supported with COMPACT STORAGE");

          stmt.comparator = CFDefinition.definitionType;
        } else {
          List<AbstractType<?>> types =
              new ArrayList<AbstractType<?>>(definedCollections == null ? 1 : 2);
          types.add(CFDefinition.definitionType);
          if (definedCollections != null)
            types.add(ColumnToCollectionType.getInstance(definedCollections));
          stmt.comparator = CompositeType.getInstance(types);
        }
      } else {
        // If we use compact storage and have only one alias, it is a
        // standard "dynamic" CF, otherwise it's a composite
        if (useCompactStorage && columnAliases.size() == 1) {
          if (definedCollections != null)
            throw new InvalidRequestException(
                "Collection types are not supported with COMPACT STORAGE");
          ColumnIdentifier alias = columnAliases.get(0);
          stmt.columnAliases.add(alias.key);
          stmt.comparator = getTypeAndRemove(stmt.columns, alias);
          if (stmt.comparator instanceof CounterColumnType)
            throw new InvalidRequestException(
                String.format("counter type is not supported for PRIMARY KEY part %s", alias));
          if (staticColumns.contains(alias))
            throw new InvalidRequestException(
                String.format("Static column %s cannot be part of the PRIMARY KEY", alias));
        } else {
          List<AbstractType<?>> types = new ArrayList<AbstractType<?>>(columnAliases.size() + 1);
          for (ColumnIdentifier t : columnAliases) {
            stmt.columnAliases.add(t.key);

            AbstractType<?> type = getTypeAndRemove(stmt.columns, t);
            if (type instanceof CounterColumnType)
              throw new InvalidRequestException(
                  String.format("counter type is not supported for PRIMARY KEY part %s", t));
            if (staticColumns.contains(t))
              throw new InvalidRequestException(
                  String.format("Static column %s cannot be part of the PRIMARY KEY", t));
            types.add(type);
          }

          if (useCompactStorage) {
            if (definedCollections != null)
              throw new InvalidRequestException(
                  "Collection types are not supported with COMPACT STORAGE");
          } else {
            // For sparse, we must add the last UTF8 component
            // and the collection type if there is one
            types.add(CFDefinition.definitionType);
            if (definedCollections != null)
              types.add(ColumnToCollectionType.getInstance(definedCollections));
          }

          if (types.isEmpty())
            throw new IllegalStateException("Nonsensical empty parameter list for CompositeType");
          stmt.comparator = CompositeType.getInstance(types);
        }
      }

      if (!staticColumns.isEmpty()) {
        // Only CQL3 tables can have static columns
        if (useCompactStorage)
          throw new InvalidRequestException(
              "Static columns are not supported in COMPACT STORAGE tables");
        // Static columns only make sense if we have at least one clustering column. Otherwise
        // everything is static anyway
        if (columnAliases.isEmpty())
          throw new InvalidRequestException(
              "Static columns are only useful (and thus allowed) if the table has at least one clustering column");
      }

      if (useCompactStorage && !stmt.columnAliases.isEmpty()) {
        if (stmt.columns.isEmpty()) {
          // The only value we'll insert will be the empty one, so the default validator don't
          // matter
          stmt.defaultValidator = BytesType.instance;
          // We need to distinguish between
          //   * I'm upgrading from thrift so the valueAlias is null
          //   * I've defined my table with only a PK (and the column value will be empty)
          // So, we use an empty valueAlias (rather than null) for the second case
          stmt.valueAlias = ByteBufferUtil.EMPTY_BYTE_BUFFER;
        } else {
          if (stmt.columns.size() > 1)
            throw new InvalidRequestException(
                String.format(
                    "COMPACT STORAGE with composite PRIMARY KEY allows no more than one column not part of the PRIMARY KEY (got: %s)",
                    StringUtils.join(stmt.columns.keySet(), ", ")));

          Map.Entry<ColumnIdentifier, AbstractType> lastEntry =
              stmt.columns.entrySet().iterator().next();
          stmt.defaultValidator = lastEntry.getValue();
          stmt.valueAlias = lastEntry.getKey().key;
          stmt.columns.remove(lastEntry.getKey());
        }
      } else {
        // For compact, we are in the "static" case, so we need at least one column defined. For
        // non-compact however, having
        // just the PK is fine since we have CQL3 row marker.
        if (useCompactStorage && stmt.columns.isEmpty())
          throw new InvalidRequestException(
              "COMPACT STORAGE with non-composite PRIMARY KEY require one column not part of the PRIMARY KEY, none given");

        // There is no way to insert/access a column that is not defined for non-compact storage, so
        // the actual validator don't matter much (except that we want to recognize counter CF as
        // limitation apply to them).
        stmt.defaultValidator =
            !stmt.columns.isEmpty()
                    && (stmt.columns.values().iterator().next() instanceof CounterColumnType)
                ? CounterColumnType.instance
                : BytesType.instance;
      }

      // If we give a clustering order, we must explicitly do so for all aliases and in the order of
      // the PK
      if (!definedOrdering.isEmpty()) {
        if (definedOrdering.size() > columnAliases.size())
          throw new InvalidRequestException(
              "Only clustering key columns can be defined in CLUSTERING ORDER directive");

        int i = 0;
        for (ColumnIdentifier id : definedOrdering.keySet()) {
          ColumnIdentifier c = columnAliases.get(i);
          if (!id.equals(c)) {
            if (definedOrdering.containsKey(c))
              throw new InvalidRequestException(
                  String.format(
                      "The order of columns in the CLUSTERING ORDER directive must be the one of the clustering key (%s must appear before %s)",
                      c, id));
            else
              throw new InvalidRequestException(
                  String.format("Missing CLUSTERING ORDER for column %s", c));
          }
          ++i;
        }
      }

      return new ParsedStatement.Prepared(stmt);
    }
Пример #12
0
public class SerializationsTest extends AbstractSerializationsTester {
  Statics statics = new Statics();

  @BeforeClass
  public static void loadSchema() throws IOException, ConfigurationException {
    loadSchema(true);
  }

  private ByteBuffer startCol = ByteBufferUtil.bytes("Start");
  private ByteBuffer stopCol = ByteBufferUtil.bytes("Stop");
  private ByteBuffer emptyCol = ByteBufferUtil.bytes("");
  public NamesQueryFilter namesPred = new NamesQueryFilter(statics.NamedCols);
  public NamesQueryFilter namesSCPred = new NamesQueryFilter(statics.NamedSCCols);
  public SliceQueryFilter emptyRangePred = new SliceQueryFilter(emptyCol, emptyCol, false, 100);
  public SliceQueryFilter nonEmptyRangePred = new SliceQueryFilter(startCol, stopCol, true, 100);
  public SliceQueryFilter nonEmptyRangeSCPred =
      new SliceQueryFilter(
          CompositeType.build(statics.SC, startCol),
          CompositeType.build(statics.SC, stopCol),
          true,
          100);

  private void testRangeSliceCommandWrite() throws IOException {
    IPartitioner part = StorageService.getPartitioner();
    AbstractBounds<RowPosition> bounds =
        new Range<Token>(part.getRandomToken(), part.getRandomToken()).toRowBounds();

    RangeSliceCommand namesCmd =
        new RangeSliceCommand(statics.KS, "Standard1", statics.readTs, namesPred, bounds, 100);
    MessageOut<RangeSliceCommand> namesCmdMsg = namesCmd.createMessage();
    RangeSliceCommand emptyRangeCmd =
        new RangeSliceCommand(statics.KS, "Standard1", statics.readTs, emptyRangePred, bounds, 100);
    MessageOut<RangeSliceCommand> emptyRangeCmdMsg = emptyRangeCmd.createMessage();
    RangeSliceCommand regRangeCmd =
        new RangeSliceCommand(
            statics.KS, "Standard1", statics.readTs, nonEmptyRangePred, bounds, 100);
    MessageOut<RangeSliceCommand> regRangeCmdMsg = regRangeCmd.createMessage();
    RangeSliceCommand namesCmdSup =
        new RangeSliceCommand(statics.KS, "Super1", statics.readTs, namesSCPred, bounds, 100);
    MessageOut<RangeSliceCommand> namesCmdSupMsg = namesCmdSup.createMessage();
    RangeSliceCommand emptyRangeCmdSup =
        new RangeSliceCommand(statics.KS, "Super1", statics.readTs, emptyRangePred, bounds, 100);
    MessageOut<RangeSliceCommand> emptyRangeCmdSupMsg = emptyRangeCmdSup.createMessage();
    RangeSliceCommand regRangeCmdSup =
        new RangeSliceCommand(
            statics.KS, "Super1", statics.readTs, nonEmptyRangeSCPred, bounds, 100);
    MessageOut<RangeSliceCommand> regRangeCmdSupMsg = regRangeCmdSup.createMessage();

    DataOutputStream out = getOutput("db.RangeSliceCommand.bin");
    namesCmdMsg.serialize(out, getVersion());
    emptyRangeCmdMsg.serialize(out, getVersion());
    regRangeCmdMsg.serialize(out, getVersion());
    namesCmdSupMsg.serialize(out, getVersion());
    emptyRangeCmdSupMsg.serialize(out, getVersion());
    regRangeCmdSupMsg.serialize(out, getVersion());
    out.close();

    // test serializedSize
    testSerializedSize(namesCmd, RangeSliceCommand.serializer);
    testSerializedSize(emptyRangeCmd, RangeSliceCommand.serializer);
    testSerializedSize(regRangeCmd, RangeSliceCommand.serializer);
    testSerializedSize(namesCmdSup, RangeSliceCommand.serializer);
    testSerializedSize(emptyRangeCmdSup, RangeSliceCommand.serializer);
    testSerializedSize(regRangeCmdSup, RangeSliceCommand.serializer);
  }

  @Test
  public void testRangeSliceCommandRead() throws IOException {
    if (EXECUTE_WRITES) testRangeSliceCommandWrite();

    DataInputStream in = getInput("db.RangeSliceCommand.bin");
    for (int i = 0; i < 6; i++) MessageIn.read(in, getVersion(), -1);
    in.close();
  }

  private void testSliceByNamesReadCommandWrite() throws IOException {
    SliceByNamesReadCommand standardCmd =
        new SliceByNamesReadCommand(
            statics.KS, statics.Key, statics.StandardCF, statics.readTs, namesPred);
    SliceByNamesReadCommand superCmd =
        new SliceByNamesReadCommand(
            statics.KS, statics.Key, statics.SuperCF, statics.readTs, namesSCPred);

    DataOutputStream out = getOutput("db.SliceByNamesReadCommand.bin");
    SliceByNamesReadCommand.serializer.serialize(standardCmd, out, getVersion());
    SliceByNamesReadCommand.serializer.serialize(superCmd, out, getVersion());
    ReadCommand.serializer.serialize(standardCmd, out, getVersion());
    ReadCommand.serializer.serialize(superCmd, out, getVersion());
    standardCmd.createMessage().serialize(out, getVersion());
    superCmd.createMessage().serialize(out, getVersion());
    out.close();

    // test serializedSize
    testSerializedSize(standardCmd, SliceByNamesReadCommand.serializer);
    testSerializedSize(superCmd, SliceByNamesReadCommand.serializer);
  }

  @Test
  public void testSliceByNamesReadCommandRead() throws IOException {
    if (EXECUTE_WRITES) testSliceByNamesReadCommandWrite();

    DataInputStream in = getInput("db.SliceByNamesReadCommand.bin");
    assert SliceByNamesReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert SliceByNamesReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    in.close();
  }

  private void testSliceFromReadCommandWrite() throws IOException {
    SliceFromReadCommand standardCmd =
        new SliceFromReadCommand(
            statics.KS, statics.Key, statics.StandardCF, statics.readTs, nonEmptyRangePred);
    SliceFromReadCommand superCmd =
        new SliceFromReadCommand(
            statics.KS, statics.Key, statics.SuperCF, statics.readTs, nonEmptyRangeSCPred);

    DataOutputStream out = getOutput("db.SliceFromReadCommand.bin");
    SliceFromReadCommand.serializer.serialize(standardCmd, out, getVersion());
    SliceFromReadCommand.serializer.serialize(superCmd, out, getVersion());
    ReadCommand.serializer.serialize(standardCmd, out, getVersion());
    ReadCommand.serializer.serialize(superCmd, out, getVersion());
    standardCmd.createMessage().serialize(out, getVersion());
    superCmd.createMessage().serialize(out, getVersion());

    out.close();

    // test serializedSize
    testSerializedSize(standardCmd, SliceFromReadCommand.serializer);
    testSerializedSize(superCmd, SliceFromReadCommand.serializer);
  }

  @Test
  public void testSliceFromReadCommandRead() throws IOException {
    if (EXECUTE_WRITES) testSliceFromReadCommandWrite();

    DataInputStream in = getInput("db.SliceFromReadCommand.bin");
    assert SliceFromReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert SliceFromReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert ReadCommand.serializer.deserialize(in, getVersion()) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    in.close();
  }

  private void testRowWrite() throws IOException {
    DataOutputStream out = getOutput("db.Row.bin");
    Row.serializer.serialize(statics.StandardRow, out, getVersion());
    Row.serializer.serialize(statics.SuperRow, out, getVersion());
    Row.serializer.serialize(statics.NullRow, out, getVersion());
    out.close();

    // test serializedSize
    testSerializedSize(statics.StandardRow, Row.serializer);
    testSerializedSize(statics.SuperRow, Row.serializer);
    testSerializedSize(statics.NullRow, Row.serializer);
  }

  @Test
  public void testRowRead() throws IOException {
    if (EXECUTE_WRITES) testRowWrite();

    DataInputStream in = getInput("db.Row.bin");
    assert Row.serializer.deserialize(in, getVersion()) != null;
    assert Row.serializer.deserialize(in, getVersion()) != null;
    assert Row.serializer.deserialize(in, getVersion()) != null;
    in.close();
  }

  private void testRowMutationWrite() throws IOException {
    RowMutation standardRowRm = new RowMutation(statics.KS, statics.StandardRow);
    RowMutation superRowRm = new RowMutation(statics.KS, statics.SuperRow);
    RowMutation standardRm = new RowMutation(statics.KS, statics.Key, statics.StandardCf);
    RowMutation superRm = new RowMutation(statics.KS, statics.Key, statics.SuperCf);
    Map<UUID, ColumnFamily> mods = new HashMap<UUID, ColumnFamily>();
    mods.put(statics.StandardCf.metadata().cfId, statics.StandardCf);
    mods.put(statics.SuperCf.metadata().cfId, statics.SuperCf);
    RowMutation mixedRm = new RowMutation(statics.KS, statics.Key, mods);

    DataOutputStream out = getOutput("db.RowMutation.bin");
    RowMutation.serializer.serialize(standardRowRm, out, getVersion());
    RowMutation.serializer.serialize(superRowRm, out, getVersion());
    RowMutation.serializer.serialize(standardRm, out, getVersion());
    RowMutation.serializer.serialize(superRm, out, getVersion());
    RowMutation.serializer.serialize(mixedRm, out, getVersion());

    standardRowRm.createMessage().serialize(out, getVersion());
    superRowRm.createMessage().serialize(out, getVersion());
    standardRm.createMessage().serialize(out, getVersion());
    superRm.createMessage().serialize(out, getVersion());
    mixedRm.createMessage().serialize(out, getVersion());

    out.close();

    // test serializedSize
    testSerializedSize(standardRowRm, RowMutation.serializer);
    testSerializedSize(superRowRm, RowMutation.serializer);
    testSerializedSize(standardRm, RowMutation.serializer);
    testSerializedSize(superRm, RowMutation.serializer);
    testSerializedSize(mixedRm, RowMutation.serializer);
  }

  @Test
  public void testRowMutationRead() throws IOException {
    // row mutation deserialization requires being able to look up the keyspace in the schema,
    // so we need to rewrite this each time.  We can go back to testing on-disk data
    // once we pull RM.keyspace field out.
    testRowMutationWrite();

    DataInputStream in = getInput("db.RowMutation.bin");
    assert RowMutation.serializer.deserialize(in, getVersion()) != null;
    assert RowMutation.serializer.deserialize(in, getVersion()) != null;
    assert RowMutation.serializer.deserialize(in, getVersion()) != null;
    assert RowMutation.serializer.deserialize(in, getVersion()) != null;
    assert RowMutation.serializer.deserialize(in, getVersion()) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;
    in.close();
  }

  private void testTruncateWrite() throws IOException {
    Truncation tr = new Truncation(statics.KS, "Doesn't Really Matter");
    TruncateResponse aff = new TruncateResponse(statics.KS, "Doesn't Matter Either", true);
    TruncateResponse neg = new TruncateResponse(statics.KS, "Still Doesn't Matter", false);
    DataOutputStream out = getOutput("db.Truncation.bin");
    Truncation.serializer.serialize(tr, out, getVersion());
    TruncateResponse.serializer.serialize(aff, out, getVersion());
    TruncateResponse.serializer.serialize(neg, out, getVersion());

    tr.createMessage().serialize(out, getVersion());
    aff.createMessage().serialize(out, getVersion());
    neg.createMessage().serialize(out, getVersion());
    // todo: notice how CF names weren't validated.
    out.close();

    // test serializedSize
    testSerializedSize(tr, Truncation.serializer);
    testSerializedSize(aff, TruncateResponse.serializer);
    testSerializedSize(neg, TruncateResponse.serializer);
  }

  @Test
  public void testTruncateRead() throws IOException {
    if (EXECUTE_WRITES) testTruncateWrite();

    DataInputStream in = getInput("db.Truncation.bin");
    assert Truncation.serializer.deserialize(in, getVersion()) != null;
    assert TruncateResponse.serializer.deserialize(in, getVersion()) != null;
    assert TruncateResponse.serializer.deserialize(in, getVersion()) != null;
    assert MessageIn.read(in, getVersion(), -1) != null;

    // set up some fake callbacks so deserialization knows that what it's deserializing is a
    // TruncateResponse
    MessagingService.instance()
        .setCallbackForTests(1, new CallbackInfo(null, null, TruncateResponse.serializer));
    MessagingService.instance()
        .setCallbackForTests(2, new CallbackInfo(null, null, TruncateResponse.serializer));

    assert MessageIn.read(in, getVersion(), 1) != null;
    assert MessageIn.read(in, getVersion(), 2) != null;
    in.close();
  }

  private void testWriteResponseWrite() throws IOException {
    WriteResponse aff = new WriteResponse();
    WriteResponse neg = new WriteResponse();
    DataOutputStream out = getOutput("db.WriteResponse.bin");
    WriteResponse.serializer.serialize(aff, out, getVersion());
    WriteResponse.serializer.serialize(neg, out, getVersion());
    out.close();

    // test serializedSize
    testSerializedSize(aff, WriteResponse.serializer);
    testSerializedSize(neg, WriteResponse.serializer);
  }

  @Test
  public void testWriteResponseRead() throws IOException {
    if (EXECUTE_WRITES) testWriteResponseWrite();

    DataInputStream in = getInput("db.WriteResponse.bin");
    assert WriteResponse.serializer.deserialize(in, getVersion()) != null;
    assert WriteResponse.serializer.deserialize(in, getVersion()) != null;
    in.close();
  }

  private static ByteBuffer bb(String s) {
    return ByteBufferUtil.bytes(s);
  }

  private static class Statics {
    private final String KS = "Keyspace1";
    private final ByteBuffer Key = ByteBufferUtil.bytes("Key01");
    private final SortedSet<ByteBuffer> NamedCols =
        new TreeSet<ByteBuffer>(BytesType.instance) {
          {
            add(ByteBufferUtil.bytes("AAA"));
            add(ByteBufferUtil.bytes("BBB"));
            add(ByteBufferUtil.bytes("CCC"));
          }
        };
    private final ByteBuffer SC = ByteBufferUtil.bytes("SCName");
    private final SortedSet<ByteBuffer> NamedSCCols =
        new TreeSet<ByteBuffer>(BytesType.instance) {
          {
            add(CompositeType.build(SC, ByteBufferUtil.bytes("AAA")));
            add(CompositeType.build(SC, ByteBufferUtil.bytes("BBB")));
            add(CompositeType.build(SC, ByteBufferUtil.bytes("CCC")));
          }
        };
    private final String StandardCF = "Standard1";
    private final String SuperCF = "Super1";

    private final long readTs = 1369935512292L;

    private final ColumnFamily StandardCf =
        TreeMapBackedSortedColumns.factory.create(KS, StandardCF);
    private final ColumnFamily SuperCf = TreeMapBackedSortedColumns.factory.create(KS, SuperCF);

    private final Row StandardRow = new Row(Util.dk("key0"), StandardCf);
    private final Row SuperRow = new Row(Util.dk("key1"), SuperCf);
    private final Row NullRow = new Row(Util.dk("key2"), null);

    private Statics() {
      StandardCf.addColumn(new Column(bb("aaaa")));
      StandardCf.addColumn(new Column(bb("bbbb"), bb("bbbbb-value")));
      StandardCf.addColumn(new Column(bb("cccc"), bb("ccccc-value"), 1000L));
      StandardCf.addColumn(new DeletedColumn(bb("dddd"), 500, 1000));
      StandardCf.addColumn(new DeletedColumn(bb("eeee"), bb("eeee-value"), 1001));
      StandardCf.addColumn(new ExpiringColumn(bb("ffff"), bb("ffff-value"), 2000, 1000));
      StandardCf.addColumn(new ExpiringColumn(bb("gggg"), bb("gggg-value"), 2001, 1000, 2002));

      SuperCf.addColumn(new Column(CompositeType.build(SC, bb("aaaa"))));
      SuperCf.addColumn(new Column(CompositeType.build(SC, bb("bbbb")), bb("bbbbb-value")));
      SuperCf.addColumn(new Column(CompositeType.build(SC, bb("cccc")), bb("ccccc-value"), 1000L));
      SuperCf.addColumn(new DeletedColumn(CompositeType.build(SC, bb("dddd")), 500, 1000));
      SuperCf.addColumn(
          new DeletedColumn(CompositeType.build(SC, bb("eeee")), bb("eeee-value"), 1001));
      SuperCf.addColumn(
          new ExpiringColumn(CompositeType.build(SC, bb("ffff")), bb("ffff-value"), 2000, 1000));
      SuperCf.addColumn(
          new ExpiringColumn(
              CompositeType.build(SC, bb("gggg")), bb("gggg-value"), 2001, 1000, 2002));
    }
  }
}
Пример #13
0
 {
   add(CompositeType.build(SC, ByteBufferUtil.bytes("AAA")));
   add(CompositeType.build(SC, ByteBufferUtil.bytes("BBB")));
   add(CompositeType.build(SC, ByteBufferUtil.bytes("CCC")));
 }