You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StefanRRichter <gi...@git.apache.org> on 2018/07/02 08:53:57 UTC

[GitHub] flink pull request #6228: [FLINK-9491] Implement timer data structure based ...

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

    https://github.com/apache/flink/pull/6228#discussion_r199426052
  
    --- Diff: flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOrderedStore.java ---
    @@ -0,0 +1,283 @@
    +/*
    + * 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.typeutils.TypeSerializer;
    +import org.apache.flink.configuration.ConfigConstants;
    +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.state.heap.CachingInternalPriorityQueueSet;
    +import org.apache.flink.util.CloseableIterator;
    +import org.apache.flink.util.FlinkRuntimeException;
    +
    +import org.rocksdb.ColumnFamilyHandle;
    +import org.rocksdb.ReadOptions;
    +import org.rocksdb.RocksDB;
    +import org.rocksdb.RocksDBException;
    +
    +import javax.annotation.Nonnegative;
    +import javax.annotation.Nonnull;
    +import javax.annotation.Nullable;
    +
    +import java.io.IOException;
    +import java.util.NoSuchElementException;
    +
    +/**
    + * Implementation of {@link org.apache.flink.runtime.state.heap.CachingInternalPriorityQueueSet.OrderedSetStore}
    + * based on RocksDB.
    + *
    + * <p>IMPORTANT: The store is ordered and the order is determined by the lexicographic order of the byte sequences
    + * produced by the provided serializer for the elements!
    + *
    + * @param <T> the type of stored elements.
    + */
    +public class RocksDBOrderedStore<T> implements CachingInternalPriorityQueueSet.OrderedSetStore<T> {
    +
    +	/** Serialized empty value to insert into RocksDB. */
    +	private static final byte[] DUMMY_BYTES = "0".getBytes(ConfigConstants.DEFAULT_CHARSET);
    +
    +	/** The RocksDB instance that serves as store. */
    +	@Nonnull
    +	private final RocksDB db;
    +
    +	/** Handle to the column family of the RocksDB instance in which the elements are stored. */
    +	@Nonnull
    +	private final ColumnFamilyHandle columnFamilyHandle;
    +
    +	/** Read options for RocksDB. */
    +	@Nonnull
    +	private final ReadOptions readOptions;
    +
    +	/**
    +	 * Serializer for the contained elements. The lexicographical order of the bytes of serialized objects must be
    +	 * aligned with their logical order.
    +	 */
    +	@Nonnull
    +	private final TypeSerializer<T> byteOrderProducingSerializer;
    +
    +	/** Wrapper to batch all writes to RocksDB. */
    +	@Nonnull
    +	private final RocksDBWriteBatchWrapper batchWrapper;
    +
    +	/** The key-group id of all elements stored in this instance. */
    +	@Nonnegative
    +	private final int keyGroupId;
    +
    +	/** The key-group id in serialized form. */
    +	@Nonnull
    +	private final byte[] groupPrefixBytes;
    +
    +	/** Output stream that helps to serialize elements. */
    +	@Nonnull
    +	private final ByteArrayOutputStreamWithPos outputStream;
    +
    +	/** Output view that helps to serialize elements, must wrap the output stream. */
    +	@Nonnull
    +	private final DataOutputViewStreamWrapper outputView;
    +
    +	public RocksDBOrderedStore(
    +		@Nonnegative int keyGroupId,
    +		@Nonnull RocksDB db,
    +		@Nonnull ColumnFamilyHandle columnFamilyHandle,
    +		@Nonnull ReadOptions readOptions,
    +		@Nonnull TypeSerializer<T> byteOrderProducingSerializer,
    +		@Nonnull ByteArrayOutputStreamWithPos outputStream,
    +		@Nonnull DataOutputViewStreamWrapper outputView,
    +		@Nonnull RocksDBWriteBatchWrapper batchWrapper) {
    +		this.db = db;
    +		this.columnFamilyHandle = columnFamilyHandle;
    +		this.readOptions = readOptions;
    +		this.byteOrderProducingSerializer = byteOrderProducingSerializer;
    +		this.outputStream = outputStream;
    +		this.outputView = outputView;
    +		this.keyGroupId = keyGroupId;
    +		this.batchWrapper = batchWrapper;
    +		this.groupPrefixBytes = createKeyGroupBytes(keyGroupId);
    +	}
    +
    +	private byte[] createKeyGroupBytes(int keyGroupId) {
    +
    +		outputStream.reset();
    +
    +		try {
    +			outputView.writeShort(keyGroupId);
    +		} catch (IOException e) {
    +			throw new FlinkRuntimeException("Could not write key-group bytes.", e);
    +		}
    +
    +		return outputStream.toByteArray();
    +	}
    +
    +	@Override
    +	public void add(@Nonnull T element) {
    +		byte[] elementBytes = serializeElement(element);
    +		try {
    +			batchWrapper.put(columnFamilyHandle, elementBytes, DUMMY_BYTES);
    +		} catch (RocksDBException e) {
    +			throw new FlinkRuntimeException("Error while getting element from RocksDB.", e);
    +		}
    +	}
    +
    +	@Override
    +	public void remove(@Nonnull T element) {
    +		byte[] elementBytes = serializeElement(element);
    +		try {
    +			batchWrapper.remove(columnFamilyHandle, elementBytes);
    +		} catch (RocksDBException e) {
    +			throw new FlinkRuntimeException("Error while removing element from RocksDB.", e);
    +		}
    +	}
    +
    +	/**
    +	 * This implementation comes at a relatively high cost per invocation. It should not be called repeatedly when it is
    +	 * clear that the value did not change. Currently this is only truly used to realize certain higher-level tests.
    +	 *
    +	 * @see org.apache.flink.runtime.state.heap.CachingInternalPriorityQueueSet.OrderedSetStore
    +	 */
    +	@Override
    +	public int size() {
    +
    +		int count = 0;
    +		try (final RocksToJavaIteratorAdapter iterator = orderedIterator()) {
    +			while (iterator.hasNext()) {
    +				iterator.next();
    +				++count;
    +			}
    +		}
    +
    +		return count;
    +	}
    +
    +	@Nonnull
    +	@Override
    +	public RocksToJavaIteratorAdapter orderedIterator() {
    +
    +		flushWriteBatch();
    +
    +		return new RocksToJavaIteratorAdapter(
    +			new RocksIteratorWrapper(
    +				db.newIterator(columnFamilyHandle, readOptions)));
    +	}
    +
    +	/**
    +	 * Ensures that recent writes are flushed and reflect in the RocksDB instance.
    +	 */
    +	private void flushWriteBatch() {
    +		try {
    +			batchWrapper.flush();
    +		} catch (RocksDBException e) {
    +			throw new FlinkRuntimeException(e);
    +		}
    +	}
    +
    +	private static boolean isPrefixWith(byte[] bytes, byte[] prefixBytes) {
    +		for (int i = 0; i < prefixBytes.length; ++i) {
    +			if (bytes[i] != prefixBytes[i]) {
    +				return false;
    +			}
    +		}
    +		return true;
    +	}
    +
    +	private byte[] serializeElement(T element) {
    +		try {
    +			outputStream.reset();
    +			outputView.writeShort(keyGroupId);
    +			byteOrderProducingSerializer.serialize(element, outputView);
    +			return outputStream.toByteArray();
    +		} catch (IOException e) {
    +			throw new FlinkRuntimeException("Error while serializing the element.", e);
    +		}
    +	}
    +
    +	private T deserializeElement(byte[] bytes) {
    +		try {
    +			// TODO introduce a stream in which we can change the internal byte[] to avoid creating instances per call
    +			ByteArrayInputStreamWithPos inputStream = new ByteArrayInputStreamWithPos(bytes);
    +			DataInputViewStreamWrapper inputView = new DataInputViewStreamWrapper(inputStream);
    +			inputView.readShort();
    --- End diff --
    
    `skip()` also has a return value that should be checked (in general, maybe not for this particular stream that we use), similar to what `skipBytes(...)`. I think still doing the check is cleaner and then I think we can also use `readShort`, and it is easier to identify as the counterpart of the corresponding `writeShort(...)`.


---