From issues-return-174287-archive-asf-public=cust-asf.ponee.io@flink.apache.org Mon Jul 2 11:50:40 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 3747B180674 for ; Mon, 2 Jul 2018 11:50:40 +0200 (CEST) Received: (qmail 75835 invoked by uid 500); 2 Jul 2018 09:50:39 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 75822 invoked by uid 99); 2 Jul 2018 09:50:39 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 02 Jul 2018 09:50:39 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2B1B9E0A8D; Mon, 2 Jul 2018 09:50:39 +0000 (UTC) From: StefanRRichter To: issues@flink.apache.org Reply-To: issues@flink.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #6228: [FLINK-9491] Implement timer data structure based ... Content-Type: text/plain Message-Id: <20180702095039.2B1B9E0A8D@git1-us-west.apache.org> Date: Mon, 2 Jul 2018 09:50:39 +0000 (UTC) Github user StefanRRichter commented on a diff in the pull request: https://github.com/apache/flink/pull/6228#discussion_r199442186 --- 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. + * + *

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 the type of stored elements. + */ +public class RocksDBOrderedStore implements CachingInternalPriorityQueueSet.OrderedSetStore { + + /** 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 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 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) { --- End diff -- I did it for simplicity, but you are right. Will change it to variable size. ---