Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 397B3200CF6 for ; Mon, 18 Sep 2017 22:33:04 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 37C621609D4; Mon, 18 Sep 2017 20:33:04 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 8008F1609DB for ; Mon, 18 Sep 2017 22:33:03 +0200 (CEST) Received: (qmail 77292 invoked by uid 500); 18 Sep 2017 20:33:02 -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 77283 invoked by uid 99); 18 Sep 2017 20:33:02 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 18 Sep 2017 20:33:02 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 3D654C06BE for ; Mon, 18 Sep 2017 20:33:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id yzpCGbG6CZl5 for ; Mon, 18 Sep 2017 20:33:01 +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 489625FB17 for ; Mon, 18 Sep 2017 20:33:01 +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 BEF13E0ECE for ; Mon, 18 Sep 2017 20:33:00 +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 307C724153 for ; Mon, 18 Sep 2017 20:33:00 +0000 (UTC) Date: Mon, 18 Sep 2017 20:33:00 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-6563) Expose time indicator attributes in the KafkaTableSource MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 18 Sep 2017 20:33:04 -0000 [ https://issues.apache.org/jira/browse/FLINK-6563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16170641#comment-16170641 ] ASF GitHub Bot commented on FLINK-6563: --------------------------------------- Github user haohui commented on a diff in the pull request: https://github.com/apache/flink/pull/4638#discussion_r139532226 --- Diff: flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java --- @@ -106,8 +240,191 @@ return deserializationSchema; } - @Override - public String explainSource() { - return ""; + /** + * Assigns ingestion time timestamps and watermarks. + */ + public static class IngestionTimeWatermarkAssigner implements AssignerWithPeriodicWatermarks { + + private long curTime = Long.MIN_VALUE; + + @Override + public long extractTimestamp(Row element, long previousElementTimestamp) { + long t = System.currentTimeMillis(); + if (t > curTime) { + curTime = t; + } + return curTime; + } + + @Nullable + @Override + public Watermark getCurrentWatermark() { + return new Watermark(curTime - 1); + } + } + + protected AssignerWithPeriodicWatermarks getAssigner() { + return this.timestampAssigner; + } + + /** + * Checks that the provided row time attribute is valid, determines its position in the schema, + * and adjusts the return type. + * + * @param rowtime The attribute to check. + */ + private void configureRowTimeAttribute(String rowtime) { + Preconditions.checkNotNull(rowtime, "Row time attribute must not be null."); + + if (this.ingestionTimeAttribute != null) { + throw new ValidationException( + "You can only specify a row time attribute OR an ingestion time attribute."); + } + + if (this.rowTimeAttribute != null) { + throw new ValidationException( + "Row time attribute can only be specified once."); + } + + // get current fields + String[] fieldNames = ((RowTypeInfo) this.getReturnType()).getFieldNames(); + TypeInformation[] fieldTypes = ((RowTypeInfo) this.getReturnType()).getFieldTypes(); + + // check if the rowtime field exists and remember position + this.rowtimeFieldPos = -1; --- End diff -- Currently in our internal system we worked around this problem by decorating the data stream return by Kafka. Having the scan operator to assign the watermark seems pretty neat. +1 for that. It might be cleaner to put both the assignment of the timestamp (i.e., `TimestampAssigner`) and the naming of the timestamp (i.e., `DefinedRowtimeAttribute` and `DefinedProctimeAttribute`) together. What do you think? > Expose time indicator attributes in the KafkaTableSource > -------------------------------------------------------- > > Key: FLINK-6563 > URL: https://issues.apache.org/jira/browse/FLINK-6563 > Project: Flink > Issue Type: Bug > Components: Table API & SQL > Reporter: Haohui Mai > Assignee: Haohui Mai > Priority: Critical > Fix For: 1.4.0 > > > This is a follow up for FLINK-5884. > After FLINK-5884 requires the {{TableSource}} interfaces to expose the processing time and the event time for the data stream. This jira proposes to expose these two information in the Kafka table source. -- This message was sent by Atlassian JIRA (v6.4.14#64029)