From issues-return-158261-archive-asf-public=cust-asf.ponee.io@flink.apache.org Wed Mar 14 03:34:07 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 3F20C18067B for ; Wed, 14 Mar 2018 03:34:07 +0100 (CET) Received: (qmail 32142 invoked by uid 500); 14 Mar 2018 02:34:06 -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 32114 invoked by uid 99); 14 Mar 2018 02:34:06 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 14 Mar 2018 02:34:06 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id CE53B1A07C5 for ; Wed, 14 Mar 2018 02:34:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -101.511 X-Spam-Level: X-Spam-Status: No, score=-101.511 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id n8Jrjiru9AFn for ; Wed, 14 Mar 2018 02:34:04 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id D69385FAE7 for ; Wed, 14 Mar 2018 02:34:03 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 57767E0CF2 for ; Wed, 14 Mar 2018 02:34:02 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 8D22921495 for ; Wed, 14 Mar 2018 02:34:00 +0000 (UTC) Date: Wed, 14 Mar 2018 02:34:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-6968) Store streaming, updating tables with unique key in queryable state MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-6968?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16398000#comment-16398000 ] ASF GitHub Bot commented on FLINK-6968: --------------------------------------- Github user xccui commented on a diff in the pull request: https://github.com/apache/flink/pull/5688#discussion_r174338947 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/QueryableTableSink.scala --- @@ -0,0 +1,175 @@ +/* + * 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.table.sinks + +import java.lang.{Boolean => JBool} + +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.functions.KeySelector +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.apache.flink.api.java.typeutils.{ResultTypeQueryable, RowTypeInfo} +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.streaming.api.functions.ProcessFunction +import org.apache.flink.table.api.StreamQueryConfig +import org.apache.flink.table.runtime.aggregate.ProcessFunctionWithCleanupState +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.slf4j.LoggerFactory + +class QueryableTableSink(private val namePrefix: String, + private val queryConfig: StreamQueryConfig) + extends UpsertStreamTableSink[Row] + with TableSinkBase[JTuple2[JBool, Row]] { + private var keys: Array[String] = _ + + /** + * Configures the unique key fields of the [[Table]] to write. + * The method is called after [[TableSink.configure()]]. + * + * The keys array might be empty, if the table consists of a single (updated) record. + * If the table does not have a key and is append-only, the keys attribute is null. + * + * @param keys the field names of the table's keys, an empty array if the table has a single + * row, and null if the table is append-only and has no key. + */ + override def setKeyFields(keys: Array[String]): Unit = { + if (keys == null) { + throw new IllegalArgumentException("keys can't be null!") + } + this.keys = keys + } + + /** + * Specifies whether the [[Table]] to write is append-only or not. + * + * @param isAppendOnly true if the table is append-only, false otherwise. + */ + override def setIsAppendOnly(isAppendOnly: JBool): Unit = { + if (isAppendOnly) { + throw new IllegalArgumentException("A QueryableTableSink can not be used with append-only tables " + + "as the table would grow infinitely") + } + } + + /** Returns the requested record type */ + override def getRecordType: TypeInformation[Row] = new RowTypeInfo(getFieldTypes, getFieldNames) + + /** Emits the DataStream. */ + override def emitDataStream(dataStream: DataStream[JTuple2[JBool, Row]]): Unit = { + val keyIndices = keys.map(getFieldNames.indexOf(_)) + val keyTypes = keyIndices.map(getFieldTypes(_)) + + val keySelectorType = new RowTypeInfo(keyTypes, keys) + + val processFunction = new QueryableStateProcessFunction( + namePrefix, + queryConfig, + keys, + getFieldNames, + getFieldTypes) + + dataStream.keyBy(new RowKeySelector(keyIndices, keySelectorType)) + .process(processFunction) --- End diff -- This `process(processFunction)` method has been deprecated. Replace it with `process(KeyedProcessFunction)`. > Store streaming, updating tables with unique key in queryable state > ------------------------------------------------------------------- > > Key: FLINK-6968 > URL: https://issues.apache.org/jira/browse/FLINK-6968 > Project: Flink > Issue Type: New Feature > Components: Table API & SQL > Reporter: Fabian Hueske > Assignee: Renjie Liu > Priority: Major > > Streaming tables with unique key are continuously updated. For example queries with a non-windowed aggregation generate such tables. Commonly, such updating tables are emitted via an upsert table sink to an external datastore (k-v store, database) to make it accessible to applications. > This issue is about adding a feature to store and maintain such a table as queryable state in Flink. By storing the table in Flnk's queryable state, we do not need an external data store to access the results of the query but can query the results directly from Flink. -- This message was sent by Atlassian JIRA (v7.6.3#76005)