Github user StefanRRichter commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3336#discussion_r102022585
  
    --- Diff: 
flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBMapState.java
 ---
    @@ -0,0 +1,579 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.MapState;
    +import org.apache.flink.api.common.state.MapStateDescriptor;
    +import org.apache.flink.api.common.typeutils.TypeSerializer;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.tuple.Tuple3;
    +import org.apache.flink.api.java.tuple.Tuple4;
    +import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
    +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
    +import org.apache.flink.core.memory.DataInputViewStreamWrapper;
    +import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
    +import 
org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
    +import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
    +import org.apache.flink.runtime.state.internal.InternalMapState;
    +import org.apache.flink.util.Preconditions;
    +import org.rocksdb.ColumnFamilyHandle;
    +import org.rocksdb.RocksDB;
    +import org.rocksdb.RocksDBException;
    +import org.rocksdb.RocksIterator;
    +import org.rocksdb.WriteOptions;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.Map;
    +
    +/**
    + * {@link MapState} implementation that stores state in RocksDB.
    + * <p>
    + * <p>{@link RocksDBStateBackend} must ensure that we set the
    + * {@link org.rocksdb.StringAppendOperator} on the column family that we 
use for our state since
    + * we use the {@code merge()} call.
    + *
    + * @param <K>  The type of the key.
    + * @param <N>  The type of the namespace.
    + * @param <UK> The type of the keys in the map state.
    + * @param <UV> The type of the values in the map state.
    + */
    +public class RocksDBMapState<K, N, UK, UV>
    +   extends AbstractRocksDBState<K, N, MapState<UK, UV>, 
MapStateDescriptor<UK, UV>, Map<UK, UV>>
    +   implements InternalMapState<N, UK, UV> {
    +
    +   /** Serializer for the keys and values */
    +   private final TypeSerializer<UK> userKeySerializer;
    +   private final TypeSerializer<UV> userValueSerializer;
    +
    +   /**
    +    * 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 RocksDBMapState}.
    +    *
    +    * @param namespaceSerializer The serializer for the namespace.
    +    * @param stateDesc The state identifier for the state.
    +    */
    +   public RocksDBMapState(ColumnFamilyHandle columnFamily,
    +                   TypeSerializer<N> namespaceSerializer,
    +                   MapStateDescriptor<UK, UV> stateDesc,
    +                   RocksDBKeyedStateBackend<K> backend) {
    +
    +           super(columnFamily, namespaceSerializer, stateDesc, backend);
    +
    +           this.userKeySerializer = stateDesc.getKeySerializer();
    +           this.userValueSerializer = stateDesc.getValueSerializer();
    +
    +           writeOptions = new WriteOptions();
    +           writeOptions.setDisableWAL(true);
    +   }
    +
    +   // 
------------------------------------------------------------------------
    +   //  MapState Implementation
    +   // 
------------------------------------------------------------------------
    +
    +   @Override
    +   public UV get(UK userKey) throws IOException {
    +           try {
    +                   byte[] rawKeyBytes = 
serializeUserKeyWithCurrentKeyAndNamespace(userKey);
    +                   byte[] rawValueBytes = backend.db.get(columnFamily, 
rawKeyBytes);
    +
    +                   return (rawValueBytes == null ? null : 
deserializeUserValue(rawValueBytes));
    +           } catch (RocksDBException e) {
    +                   throw new RuntimeException("Error while getting data 
from RocksDB.", e);
    +           }
    +   }
    +
    +   @Override
    +   public void put(UK userKey, UV userValue) throws IOException {
    +           if (userValue == null) {
    +                   remove(userKey);
    +           }
    +
    +           try {
    +                   byte[] rawKeyBytes = 
serializeUserKeyWithCurrentKeyAndNamespace(userKey);
    +                   byte[] rawValueBytes = serializeUserValue(userValue);
    +
    +                   backend.db.put(columnFamily, writeOptions, rawKeyBytes, 
rawValueBytes);
    +           } catch (RocksDBException e) {
    +                   throw new RuntimeException("Error while putting data 
into RocksDB", e);
    +           }
    +   }
    +
    +   @Override
    +   public void remove(UK userKey) throws IOException {
    +           try {
    +                   byte[] rawKeyBytes = 
serializeUserKeyWithCurrentKeyAndNamespace(userKey);
    +
    +                   backend.db.remove(columnFamily, writeOptions, 
rawKeyBytes);
    +           } catch (RocksDBException e) {
    +                   throw new RuntimeException("Error while removing data 
from RocksDB.", e);
    +           }
    +   }
    +
    +   @Override
    +   public boolean contains(UK userKey) throws IOException {
    +           try {
    +                   byte[] rawKeyBytes = 
serializeUserKeyWithCurrentKeyAndNamespace(userKey);
    +                   byte[] rawValueBytes = backend.db.get(columnFamily, 
rawKeyBytes);
    +
    +                   return (rawValueBytes != null);
    +           } catch (RocksDBException e) {
    +                   throw new RuntimeException("Error while getting data 
from RocksDB", e);
    +           }
    +   }
    +
    +   @Override
    +   public int size() throws IOException {
    +           Iterator<Map.Entry<UK, UV>> iterator = iterator();
    +
    +           int count = 0;
    +           while (iterator.hasNext()) {
    +                   count++;
    +                   iterator.next();
    +           }
    +
    +           return count;
    +   }
    +
    +   @Override
    +   public Iterable<UK> keys() {
    +           return new Iterable<UK>() {
    +                   @Override
    +                   public Iterator<UK> iterator() {
    +                           return new RocksDBMapIterator<UK>(backend.db, 
serializeCurrentKey()) {
    +                                   @Override
    +                                   public UK next() {
    +                                           RocksDBMapEntry entry = 
nextEntry();
    +                                           return (entry == null ? null : 
entry.getKey());
    +                                   }
    +                           };
    +                   }
    +           };
    +   }
    +
    +   @Override
    +   public Iterable<UV> values() {
    +           return new Iterable<UV>() {
    +                   @Override
    +                   public Iterator<UV> iterator() {
    +                           return new RocksDBMapIterator<UV>(backend.db, 
serializeCurrentKey()) {
    +                                   @Override
    +                                   public UV next() {
    +                                           RocksDBMapEntry entry = 
nextEntry();
    +                                           return (entry == null ? null : 
entry.getValue());
    +                                   }
    +                           };
    +                   }
    +           };
    +   }
    +
    +   @Override
    +   public Iterator<Map.Entry<UK, UV>> iterator() {
    +           return new RocksDBMapIterator<Map.Entry<UK, UV>>(backend.db, 
serializeCurrentKey()) {
    +                   @Override
    +                   public Map.Entry<UK, UV> next() {
    +                           return nextEntry();
    +                   }
    +           };
    +   }
    +
    +   @Override
    +   public Iterable<Map.Entry<UK, UV>> get() throws Exception {
    +           final Iterator<Map.Entry<UK, UV>> iterator = iterator();
    +
    +           // Return null to make the behavior consistent with other 
states.
    +           if (!iterator.hasNext()) {
    +                   return null;
    +           } else {
    +                   return new Iterable<Map.Entry<UK, UV>>() {
    +                           @Override
    +                           public Iterator<Map.Entry<UK, UV>> iterator() {
    +                                   return iterator;
    +                           }
    +                   };
    +           }
    +   }
    +
    +   @Override
    +   public void add(Map<UK, UV> map) throws Exception {
    +           if (map == null) {
    +                   return;
    +           }
    +           
    +           for (Map.Entry<UK, UV> entry : map.entrySet()) {
    +                   put(entry.getKey(), entry.getValue());
    +           }
    +   }
    +   
    +   @Override
    +   public void clear() {
    +           Iterator<Map.Entry<UK, UV>> iterator = iterator();
    +           
    +           while (iterator.hasNext()) {
    +                   iterator.next();
    +                   iterator.remove();
    +           }
    +   }
    +   
    +   @Override
    +   @SuppressWarnings("unchecked")
    +   public byte[] getSerializedValue(byte[] serializedKeyAndNamespace) 
throws Exception {
    +           Preconditions.checkNotNull(serializedKeyAndNamespace, 
"Serialized key and namespace");
    +
    +           //TODO make KvStateRequestSerializer key-group aware to save 
this round trip and key-group computation
    +           Tuple2<K, N> des = 
KvStateRequestSerializer.deserializeKeyAndNamespace(
    +                           serializedKeyAndNamespace,
    +                           backend.getKeySerializer(),
    +                           namespaceSerializer);
    +
    +           int keyGroup = KeyGroupRangeAssignment.assignToKeyGroup(des.f0, 
backend.getNumberOfKeyGroups());
    +           
    +           ByteArrayOutputStreamWithPos outputStream = new 
ByteArrayOutputStreamWithPos(128);
    +           DataOutputViewStreamWrapper outputView = new 
DataOutputViewStreamWrapper(outputStream);
    +
    +           writeKeyWithGroupAndNamespace(keyGroup, des.f0, des.f1, 
outputStream, outputView);
    +           byte[] keyPrefixBytes = outputStream.toByteArray();
    +
    +           Iterator<Map.Entry<UK, UV>> iterator = new 
RocksDBMapIterator<Map.Entry<UK, UV>>(backend.db, keyPrefixBytes) {
    +                   @Override
    +                   public Map.Entry<UK, UV> next() {
    +                           return nextEntry();
    +                   }
    +           };
    +
    +           // Return null to make the behavior consistent
    +           if (!iterator.hasNext()) {
    +                   return null;
    +           }
    +           
    +           outputStream.reset();
    +           
    +           while (iterator.hasNext()) {
    +                   Map.Entry<UK, UV> entry = iterator.next();
    +                   
    +                   userKeySerializer.serialize(entry.getKey(), outputView);
    +                   userValueSerializer.serialize(entry.getValue(), 
outputView);
    +           }
    +           
    +           return outputStream.toByteArray();
    +   }
    +   
    +   // 
------------------------------------------------------------------------
    +   //  Serialization Methods
    +   // 
------------------------------------------------------------------------
    +   
    +   private byte[] serializeCurrentKey() {
    +           try {
    +                   writeCurrentKeyWithGroupAndNamespace();
    +                   
    +                   return keySerializationStream.toByteArray();
    +           } catch (IOException e) {
    +                   throw new RuntimeException("Error while serializing the 
current key.");
    +           }
    +   }
    +
    +   private byte[] serializeUserKeyWithCurrentKeyAndNamespace(UK userKey) {
    +           try {
    +                   writeCurrentKeyWithGroupAndNamespace();
    +                   userKeySerializer.serialize(userKey, 
keySerializationDataOutputView);
    +                   
    +                   return keySerializationStream.toByteArray();
    +           } catch (IOException e) {
    +                   throw new RuntimeException("Error while serializing the 
user key.", e);
    +           }
    +   }
    +   
    +   private byte[] serializeUserKey(int keyGroup, K key, N namespace, UK 
userKey) {
    +           try {
    +                   writeKeyWithGroupAndNamespace(keyGroup, key, namespace, 
keySerializationStream, keySerializationDataOutputView);
    +                   userKeySerializer.serialize(userKey, 
keySerializationDataOutputView);
    +                   
    +                   return keySerializationStream.toByteArray();
    +           } catch (IOException e) {
    +                   throw new RuntimeException("Error while serializing the 
user key.", e);
    +           }
    +   }
    +
    +   private byte[] serializeUserValue(UV userValue) {
    +           try {
    +                   keySerializationStream.reset();
    +                   userValueSerializer.serialize(userValue, 
keySerializationDataOutputView);
    +                   
    +                   return keySerializationStream.toByteArray();
    +           } catch (IOException e) {
    +                   throw new RuntimeException("Error while serializing the 
user value.", e);
    +           }
    +   }
    +
    +   private Tuple4<Integer, K, N, UK> deserializeUserKey(byte[] 
rawKeyBytes) {
    --- End diff --
    
    As far as I can see, all caller of this method are only interested in the 
user key. We could avoid some object creation overhead, if we only focus on 
returning the user key, in particular because this method is called very often. 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to