flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4856) Add MapState for keyed streams
Date Tue, 21 Feb 2017 03:45:45 GMT

    [ https://issues.apache.org/jira/browse/FLINK-4856?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15875320#comment-15875320
] 

ASF GitHub Bot commented on FLINK-4856:
---------------------------------------

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

    https://github.com/apache/flink/pull/3336#discussion_r102127767
  
    --- 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) {
    +		try {
    +			ByteArrayInputStreamWithPos bais = new ByteArrayInputStreamWithPos(rawKeyBytes);
    +			DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais);
    +
    +			Tuple3<Integer, K, N> keyAndNamespace = readKeyWithGroupAndNamespace(bais, in);
    +			UK userKey = userKeySerializer.deserialize(in);
    +
    +			return new Tuple4<>(keyAndNamespace.f0, keyAndNamespace.f1, keyAndNamespace.f2,
userKey);
    +		} catch (IOException e) {
    +			throw new RuntimeException("Error while deserializing the user key.", e);
    +		}
    +	}
    +
    +	private UV deserializeUserValue(byte[] rawValueBytes) {
    +		try {
    +			ByteArrayInputStreamWithPos bais = new ByteArrayInputStreamWithPos(rawValueBytes);
    +			DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais);
    +
    +			return userValueSerializer.deserialize(in);
    +		} catch (IOException e) {
    +			throw new RuntimeException("Error while deserializing the user value.", e);
    +		}
    +	}
    +	
    +	// ------------------------------------------------------------------------
    +	//  Internal Classes
    +	// ------------------------------------------------------------------------
    +	
    +	/** A map entry in RocksDBMapState */
    +	private class RocksDBMapEntry implements Map.Entry<UK, UV> {
    +		private final RocksDB db;
    +		
    +		/** The raw bytes of the key stored in RocksDB. Each user key is stored in RocksDB
    +		 * with the format #KeyGroup#Key#Namespace#UserKey. */
    +		private final byte[] rawKeyBytes;
    +		
    +		/** The raw bytes of the value stored in RocksDB */
    +		private final byte[] rawValueBytes;
    +
    +		/** True if the entry has been deleted. */
    +		private boolean deleted;
    +
    +		/** The user key and value. The deserialization is performed lazily, i.e. the key
    +		 * and the value is deserialized only when they are accessed. */
    +		private UK userKey = null;
    +		private UV userValue = null;
    +
    +		RocksDBMapEntry(final RocksDB db, final byte[] rawKeyBytes, final byte[] rawValueBytes)
{
    +			this.db = db;
    +			
    +			this.rawKeyBytes = rawKeyBytes;
    +			this.rawValueBytes = rawValueBytes;
    +			this.deleted = false;
    +		}
    +		
    +		@Override
    +		public UK getKey() {
    +			if (userKey == null) {
    +				userKey = deserializeUserKey(rawKeyBytes).f3;
    +			}
    +
    +			return userKey;
    +		}
    +
    +		@Override
    +		public UV getValue() {
    +			if (deleted) {
    +				return null;
    +			} else {
    +				if (userValue == null) {
    +					userValue = deserializeUserValue(rawValueBytes);
    --- End diff --
    
    Good point. I will update it as suggested. 


> Add MapState for keyed streams
> ------------------------------
>
>                 Key: FLINK-4856
>                 URL: https://issues.apache.org/jira/browse/FLINK-4856
>             Project: Flink
>          Issue Type: New Feature
>          Components: DataStream API, State Backends, Checkpointing
>            Reporter: Xiaogang Shi
>            Assignee: Xiaogang Shi
>
> Many states in keyed streams are organized as key-value pairs. Currently, these states
are implemented by storing the entire map into a ValueState or a ListState. The implementation
however is very costly because all entries have to be serialized/deserialized when updating
a single entry. To improve the efficiency of these states, MapStates are urgently needed.




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Mime
View raw message