/*
* 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.flink.contrib.streaming.state;
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputViewStreamWrapper;
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
import org.apache.flink.runtime.state.internal.InternalValueState;
import org.rocksdb.ColumnFamilyHandle;
import org.rocksdb.RocksDBException;
import org.rocksdb.WriteOptions;
import java.io.ByteArrayInputStream;
import java.io.IOException;
/**
* {@link ValueState} implementation that stores state in RocksDB.
*
* @param <K> The type of the key.
* @param <N> The type of the namespace.
* @param <V> The type of value that the state state stores.
*/
public class RocksDBValueState<K, N, V>
extends AbstractRocksDBState<K, N, ValueState<V>, ValueStateDescriptor<V>, V>
implements InternalValueState<N, V> {
/** Serializer for the values */
private final TypeSerializer<V> valueSerializer;
/**
* We disable writes to the write-ahead-log here. We can't have these in the base class
* because JNI segfaults for some reason if they are.
*/
private final WriteOptions writeOptions;
/**
* Creates a new {@code RocksDBValueState}.
*
* @param namespaceSerializer The serializer for the namespace.
* @param stateDesc The state identifier for the state. This contains name
* and can create a default state value.
*/
public RocksDBValueState(ColumnFamilyHandle columnFamily,
TypeSerializer<N> namespaceSerializer,
ValueStateDescriptor<V> stateDesc,
RocksDBKeyedStateBackend<K> backend) {
super(columnFamily, namespaceSerializer, stateDesc, backend);
this.valueSerializer = stateDesc.getSerializer();
writeOptions = new WriteOptions();
writeOptions.setDisableWAL(true);
}
@Override
public V value() {
try {
writeCurrentKeyWithGroupAndNamespace();
byte[] key = keySerializationStream.toByteArray();
byte[] valueBytes = backend.db.get(columnFamily, key);
if (valueBytes == null) {
return stateDesc.getDefaultValue();
}
return valueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStream(valueBytes)));
} catch (IOException|RocksDBException e) {
throw new RuntimeException("Error while retrieving data from RocksDB.", e);
}
}
@Override
public void update(V value) throws IOException {
if (value == null) {
clear();
return;
}
DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(keySerializationStream);
try {
writeCurrentKeyWithGroupAndNamespace();
byte[] key = keySerializationStream.toByteArray();
keySerializationStream.reset();
valueSerializer.serialize(value, out);
backend.db.put(columnFamily, writeOptions, key, keySerializationStream.toByteArray());
} catch (Exception e) {
throw new RuntimeException("Error while adding data to RocksDB", e);
}
}
}