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 Mon, 20 Feb 2017 14:08:46 GMT

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

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

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

    https://github.com/apache/flink/pull/3336#discussion_r102018529
  
    --- 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);
    --- End diff --
    
    Again, I suggest to use a more specific exception type over runtime exception. There are
a couple more cases in this class.


> 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