Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

CASSANDRA-19516 #3247

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
@@ -0,0 +1,57 @@
/*
* 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.cql3.functions;

import java.nio.ByteBuffer;

import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.tcm.Transformation;
import org.apache.cassandra.utils.ByteBufferUtil;

public class ClusterMetadataFcts
{
public static void addFunctionsTo(NativeFunctions functions)
{
functions.add(transformationKind);
}

public static final NativeScalarFunction transformationKind = new TransformationKind();
private static final class TransformationKind extends NativeScalarFunction
{

private TransformationKind()
{
super("transformation_kind", UTF8Type.instance, Int32Type.instance);
}

@Override
public ByteBuffer execute(Arguments arguments) throws InvalidRequestException
{
Number id = arguments.get(0);
if (id.intValue() < 0 || id.intValue() > Transformation.Kind.values().length -1)
throw new InvalidRequestException(id + " is not a valid Transformation.Kind id");

Transformation.Kind kind = Transformation.Kind.fromId(id.intValue());
return ByteBufferUtil.bytes(kind.name());

}
}
}
Expand Up @@ -47,6 +47,7 @@ public class NativeFunctions
MathFcts.addFunctionsTo(this);
MaskingFcts.addFunctionsTo(this);
VectorFcts.addFunctionsTo(this);
ClusterMetadataFcts.addFunctionsTo(this);
}
};

Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/db/SystemKeyspace.java
Expand Up @@ -489,7 +489,7 @@ private SystemKeyspace()
+ "epoch bigint,"
+ "entry_id bigint,"
+ "transformation blob,"
+ "kind text,"
+ "kind int,"
+ "PRIMARY KEY (epoch))")
.partitioner(MetaStrategy.partitioner)
.compaction(CompactionParams.twcs(ImmutableMap.of("compaction_window_unit","DAYS",
Expand Down
Expand Up @@ -66,7 +66,7 @@ public DataSet data()
"FROM %s.%s", METADATA_KEYSPACE_NAME, TABLE_NAME), ConsistencyLevel.QUORUM);
for (UntypedResultSet.Row r : res)
{
Transformation.Kind kind = Transformation.Kind.valueOf(r.getString("kind"));
Transformation.Kind kind = Transformation.Kind.fromId(r.getInt("kind"));
Transformation transformation = kind.fromVersionedBytes(r.getBlob("transformation"));

result.row(r.getLong("epoch"))
Expand Down
Expand Up @@ -68,7 +68,7 @@ private DistributedMetadataLogKeyspace(){}
+ "epoch bigint,"
+ "entry_id bigint,"
+ "transformation blob,"
+ "kind text,"
+ "kind int,"
+ "PRIMARY KEY (epoch))";

public static final TableMetadata Log =
Expand All @@ -88,7 +88,7 @@ public static boolean initialize() throws IOException
UntypedResultSet result = QueryProcessor.execute(init, ConsistencyLevel.QUORUM,
FIRST.getEpoch(),
Transformation.Kind.PRE_INITIALIZE_CMS.toVersionedBytes(PreInitialize.blank()),
Transformation.Kind.PRE_INITIALIZE_CMS.toString(),
Transformation.Kind.PRE_INITIALIZE_CMS.id,
Entry.Id.NONE.entryId);

UntypedResultSet.Row row = result.one();
Expand All @@ -97,7 +97,7 @@ public static boolean initialize() throws IOException

if (row.getLong("epoch") == FIRST.getEpoch() &&
row.getLong("entry_id") == Entry.Id.NONE.entryId &&
Transformation.Kind.PRE_INITIALIZE_CMS.toString().equals(row.getString("kind")))
Transformation.Kind.PRE_INITIALIZE_CMS.id == row.getInt("kind"))
return true;

throw new IllegalStateException("Could not initialize log.");
Expand Down Expand Up @@ -137,7 +137,7 @@ public static boolean tryCommit(Entry.Id entryId,
nextEpoch.getEpoch(),
entryId.entryId,
serializedEvent,
transform.kind().toString());
transform.kind().id);

return result.one().getBoolean("[applied]");
}
Expand Down Expand Up @@ -191,7 +191,7 @@ public EntryHolder getEntries(Epoch since) throws IOException
{
long entryId = row.getLong("entry_id");
Epoch epoch = Epoch.create(row.getLong("epoch"));
Transformation.Kind kind = Transformation.Kind.valueOf(row.getString("kind"));
Transformation.Kind kind = Transformation.Kind.fromId(row.getInt("kind"));
Transformation transform = kind.fromVersionedBytes(row.getBlob("transformation"));
entryHolder.add(new Entry(new Entry.Id(entryId), epoch, transform));
}
Expand Down
2 changes: 1 addition & 1 deletion src/java/org/apache/cassandra/tcm/Transformation.java
Expand Up @@ -217,7 +217,7 @@ enum Kind
;

private final Supplier<AsymmetricMetadataSerializer<Transformation, ? extends Transformation>> serializer;
private final int id;
public final int id;

private static final Kind[] idToKindMap;

Expand Down
Expand Up @@ -75,7 +75,7 @@ public void append(Entry entry)
ByteBuffer serializedTransformation = entry.transform.kind().toVersionedBytes(entry.transform);
String query = String.format("INSERT INTO %s.%s (epoch, entry_id, transformation, kind) VALUES (?,?,?,?)",
SchemaConstants.SYSTEM_KEYSPACE_NAME, NAME);
executeInternal(query, entry.epoch.getEpoch(), entry.id.entryId, serializedTransformation, entry.transform.kind().toString());
executeInternal(query, entry.epoch.getEpoch(), entry.id.entryId, serializedTransformation, entry.transform.kind().id);
// todo; should probably not flush every time, but it simplifies tests
Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(NAME).forceBlockingFlush(ColumnFamilyStore.FlushReason.INTERNALLY_FORCED);
}
Expand Down Expand Up @@ -149,7 +149,7 @@ private static EntryHolder toEntryHolder(Epoch since, UntypedResultSet resultSet
{
long entryId = row.getLong("entry_id");
Epoch epoch = Epoch.create(row.getLong("epoch"));
Transformation.Kind kind = Transformation.Kind.valueOf(row.getString("kind"));
Transformation.Kind kind = Transformation.Kind.fromId(row.getInt("kind"));
Transformation transform = kind.fromVersionedBytes(row.getBlob("transformation"));
holder.add(new Entry(new Entry.Id(entryId), epoch, transform));
}
Expand Down
@@ -0,0 +1,68 @@
/*
* 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.cql3.functions;

import java.nio.charset.CharacterCodingException;

import org.junit.Test;

import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.exceptions.InvalidRequestException;
import org.apache.cassandra.tcm.Transformation;
import org.apache.cassandra.transport.ProtocolVersion;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.assertj.core.api.Assertions;

import static org.apache.cassandra.cql3.functions.ClusterMetadataFcts.transformationKind;
import static org.junit.Assert.assertEquals;

public class ClusterMetadataFctsTest
{

@Test
public void testTransformationKind() throws CharacterCodingException
{
int max = -1;
for (Transformation.Kind kind : Transformation.Kind.values())
{
Arguments arguments = transformationKind.newArguments(ProtocolVersion.CURRENT);
arguments.set(0, Int32Type.instance.decompose(kind.id));
assertEquals(kind.name(), ByteBufferUtil.string(transformationKind.execute(arguments)));
if (kind.id > max)
max = kind.id;
}

for (int boundary : new int[]{-1, max+1})
{
Arguments arguments = transformationKind.newArguments(ProtocolVersion.CURRENT);
arguments.set(0, Int32Type.instance.decompose(boundary));
try
{
transformationKind.execute(arguments);
}
catch (Exception e)
{
Assertions.assertThat(e)
.isInstanceOf(InvalidRequestException.class)
.hasMessageContaining(boundary + " is not a valid Transformation.Kind id");
}
}

}
}