Skip to content

Commit

Permalink
STORM-2492: Adding Cassandra Object Mapper statement builder.
Browse files Browse the repository at this point in the history
Allow bolts to emit object annotated with `@Table` and generate CQL for them automatically using the Cassandra object mapper.
  • Loading branch information
tandrup committed Apr 25, 2017
1 parent 7e183c0 commit 267288c
Show file tree
Hide file tree
Showing 5 changed files with 299 additions and 0 deletions.
34 changes: 34 additions & 0 deletions external/storm-cassandra/README.md
Expand Up @@ -142,6 +142,40 @@ import static org.apache.storm.cassandra.DynamicStatementBuilder.*
);
```

#### Writing using Cassandra Object Mapper

Instead of defining CQL statements by hand, it is possible to define CQL using cassandra object mapper.

In the topology we need to define what fields in the tuple will hold the operation (INSERT/DELETE) and the actual value:

```java
new CassandraWriterBolt(new ObjectMapperCQLStatementMapperBuilder("operation", "model"))
```

Define some class using object mapper:

```java
@Table(keyspace = "my_keyspace", name = "my_table")
public class ValueObject {
...
}
```

And in the bolt that emits to the cassandra bolt:

```java
collector.emit(new Values(ObjectMapperOperation.SAVE, new ValueObject("foo", "bar")));

```
##### Custom codecs

To add custom type codes to the mapping you need to define a lambda expression to work around TypeCodecs not being serializable:

```java
new ObjectMapperCQLStatementMapperBuilder("operation", "model")
.withCodecs(Arrays.asList(() -> new EnumNameCodec<>(MyEnum.class)));
```
### How to handle query execution results

The interface *ExecutionResultHandler* can be used to custom how an execution result should be handle.
Expand Down
7 changes: 7 additions & 0 deletions external/storm-cassandra/pom.xml
Expand Up @@ -70,6 +70,13 @@
<scope>provided</scope>
</dependency>

<dependency>
<groupId>com.datastax.cassandra</groupId>
<artifactId>cassandra-driver-mapping</artifactId>
<version>${cassandra.driver.core.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.cassandra</groupId>
<artifactId>cassandra-all</artifactId>
Expand Down
@@ -0,0 +1,36 @@
/**
* 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.storm.cassandra.query;

public enum ObjectMapperOperation {
/**
* Save all fields from the object in the value field
*/
SAVE,
/**
* Save all non-null fields from the object in the value field
*/
SAVE_IGNORE_NULLS,
/**
* Delete the object in the value field.
*
* Only the primary key fields needs to be set.
*/
DELETE
}
@@ -0,0 +1,91 @@
/**
* 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.storm.cassandra.query.builder;

import com.datastax.driver.core.TypeCodec;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.stream.Collectors;
import org.apache.storm.cassandra.query.CQLStatementBuilder;
import org.apache.storm.cassandra.query.impl.ObjectMapperCQLStatementMapper;

/**
* Builds a tuple mapper that is able to map objects annotated with {@link com.datastax.driver.mapping.annotations.Table} to CQL statements.
*
* Needs at least <pre>operationField</pre> and <pre>valueField</pre>.
*
* Writetime, TTL and consistency level can be specified in optional tuple fields.
*/
public class ObjectMapperCQLStatementMapperBuilder implements CQLStatementBuilder<ObjectMapperCQLStatementMapper>, Serializable {
private final String operationField;
private final String valueField;
private final Collection<Class<?>> udtClasses = new ArrayList<>();
private final Collection<SerializableCallable<TypeCodec<?>>> codecProducers = new ArrayList<>();
private String timestampField;
private String ttlField;
private String consistencyLevelField;

public ObjectMapperCQLStatementMapperBuilder(String operationField, String valueField) {
this.operationField = operationField;
this.valueField = valueField;
}

public ObjectMapperCQLStatementMapper build() {
List<TypeCodec<?>> codecs = codecProducers.stream().map(codecProducer -> {
try {
return codecProducer.call();
} catch (Exception e) {
throw new RuntimeException(e);
}
}).collect(Collectors.toList());
return new ObjectMapperCQLStatementMapper(operationField, valueField, timestampField, ttlField, consistencyLevelField, codecs, udtClasses);
}

public ObjectMapperCQLStatementMapperBuilder withCodecs(List<SerializableCallable<TypeCodec<?>>> codecProducer) {
this.codecProducers.addAll(codecProducer);
return this;
}

public ObjectMapperCQLStatementMapperBuilder withUDTCodecs(List<Class<?>> udtClass) {
this.udtClasses.addAll(udtClass);
return this;
}

public ObjectMapperCQLStatementMapperBuilder withTimestampField(String timestampField) {
this.timestampField = timestampField;
return this;
}

public ObjectMapperCQLStatementMapperBuilder withTtlField(String ttlField) {
this.ttlField = ttlField;
return this;
}

public ObjectMapperCQLStatementMapperBuilder withConsistencyLevelField(String consistencyLevelField) {
this.consistencyLevelField = consistencyLevelField;
return this;
}

public interface SerializableCallable<R> extends Callable<R>, Serializable {
}
}

@@ -0,0 +1,131 @@
/**
* 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.storm.cassandra.query.impl;

import com.datastax.driver.core.CodecRegistry;
import com.datastax.driver.core.ConsistencyLevel;
import com.datastax.driver.core.Session;
import com.datastax.driver.core.Statement;
import com.datastax.driver.core.TypeCodec;
import com.datastax.driver.mapping.Mapper;
import com.datastax.driver.mapping.Mapper.Option;
import com.datastax.driver.mapping.MappingManager;
import com.google.common.base.Preconditions;
import java.time.Instant;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.WeakHashMap;
import org.apache.storm.cassandra.query.CQLStatementTupleMapper;
import org.apache.storm.cassandra.query.ObjectMapperOperation;
import org.apache.storm.tuple.ITuple;

/**
* Tuple mapper that is able to map objects annotated with {@link com.datastax.driver.mapping.annotations.Table} to CQL statements
*/
public class ObjectMapperCQLStatementMapper implements CQLStatementTupleMapper {
private static final Map<Session, MappingManager> mappingManagers = new WeakHashMap<>();

private final String operationField;
private final String valueField;
private final String timestampField;
private final String ttlField;
private final String consistencyLevelField;
private final Collection<TypeCodec<?>> codecs;
private final Collection<Class<?>> udtClasses;

public ObjectMapperCQLStatementMapper(String operationField, String valueField, String timestampField, String ttlField, String consistencyLevelField, Collection<TypeCodec<?>> codecs, Collection<Class<?>> udtClasses) {
Preconditions.checkNotNull(operationField, "Operation field must not be null");
Preconditions.checkNotNull(valueField, "Value field should not be null");
this.operationField = operationField;
this.valueField = valueField;
this.timestampField = timestampField;
this.ttlField = ttlField;
this.consistencyLevelField = consistencyLevelField;
this.codecs = codecs;
this.udtClasses = udtClasses;
}

@Override
public List<Statement> map(Map map, Session session, ITuple tuple) {
final ObjectMapperOperation operation = (ObjectMapperOperation)tuple.getValueByField(operationField);

Preconditions.checkNotNull(operation, "Operation must not be null");

final Object value = tuple.getValueByField(valueField);
final Object timestampObject = timestampField != null ? tuple.getValueByField(timestampField) : null;
final Number ttl = ttlField != null ? (Number) tuple.getValueByField(ttlField) : null;
final ConsistencyLevel consistencyLevel = consistencyLevelField != null ? (ConsistencyLevel) tuple.getValueByField(consistencyLevelField) : null;

final Class<?> valueClass = value.getClass();

final Mapper mapper = getMappingManager(session).mapper(valueClass);

Collection<Option> options = new ArrayList<>();

if (timestampObject != null) {
if (timestampObject instanceof Number) {
options.add(Option.timestamp(((Number) timestampObject).longValue()));
} else if (timestampObject instanceof Instant) {
Instant timestamp = (Instant) timestampObject;
options.add(Option.timestamp(timestamp.getEpochSecond() * 1000_0000l + timestamp.getNano() / 1000l));
}
}

if (ttl != null) {
options.add(Option.ttl(ttl.intValue()));
}

if (consistencyLevel != null) {
options.add(Option.consistencyLevel(consistencyLevel));
}

if (operation == ObjectMapperOperation.SAVE) {
options.add(Option.saveNullFields(true));
return Arrays.asList(mapper.saveQuery(value, options.toArray(new Option[options.size()])));
} else if (operation == ObjectMapperOperation.SAVE_IGNORE_NULLS) {
options.add(Option.saveNullFields(false));
return Arrays.asList(mapper.saveQuery(value, options.toArray(new Option[options.size()])));
} else if (operation == ObjectMapperOperation.DELETE) {
return Arrays.asList(mapper.deleteQuery(value, options.toArray(new Option[options.size()])));
} else {
throw new UnsupportedOperationException("Unknown operation: " + operation);
}
}

private MappingManager getMappingManager(Session session) {
synchronized (mappingManagers) {
MappingManager mappingManager = mappingManagers.get(session);
if (mappingManager == null) {
mappingManager = new MappingManager(session);
mappingManagers.put(session, mappingManager);
CodecRegistry codecRegistry = session.getCluster().getConfiguration().getCodecRegistry();
for (TypeCodec<?> codec : codecs) {
codecRegistry.register(codec);
}
for (Class<?> udtClass : udtClasses) {
mappingManager.udtCodec(udtClass);
}
}
return mappingManager;
}
}
}

0 comments on commit 267288c

Please sign in to comment.