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 15BAA200D03 for ; Sat, 9 Sep 2017 09:24:06 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 144651609CF; Sat, 9 Sep 2017 07:24:06 +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 328691609C0 for ; Sat, 9 Sep 2017 09:24:05 +0200 (CEST) Received: (qmail 53761 invoked by uid 500); 9 Sep 2017 07:24:03 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 53740 invoked by uid 99); 9 Sep 2017 07:24:03 -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; Sat, 09 Sep 2017 07:24:03 +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 D0EE61A58C6 for ; Sat, 9 Sep 2017 07:24:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.201 X-Spam-Level: X-Spam-Status: No, score=-99.201 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, LOTS_OF_MONEY=0.001, 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 (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id a3b43XAUaAQn for ; Sat, 9 Sep 2017 07:24: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 EE2C960FA3 for ; Sat, 9 Sep 2017 07:24:00 +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 6D282E00C9 for ; Sat, 9 Sep 2017 07:24: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 2C7EB24147 for ; Sat, 9 Sep 2017 07:24:00 +0000 (UTC) Date: Sat, 9 Sep 2017 07:24:00 +0000 (UTC) From: "Steven Jon Anderson (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (BEAM-2870) BQ Partitioned Table Write Fails When Destination has Partition Decorator MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Sat, 09 Sep 2017 07:24:06 -0000 [ https://issues.apache.org/jira/browse/BEAM-2870?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Steven Jon Anderson updated BEAM-2870: -------------------------------------- Description: Dataflow Job ID: https://console.cloud.google.com/dataflow/job/2017-09-08_23_03_14-14637186041605198816 Tagging [~reuvenlax] as I believe he built the time partitioning integration that was merged into master. *Background* Our production pipeline ingests millions of events per day and routes events into our clients' numerous tables. To keep costs down, all of our tables are partitioned. However, this requires that we create the tables before we allow events to process as creating partitioned tables isn't supported in 2.1.0. We've been looking forward to [~reuvenlax]'s partition table write feature ([#3663|https://github.com/apache/beam/pull/3663]) to get merged into master for some time now as it'll allow us to launch our client platforms much, much faster. Today we got around to testing the 2.2.0 nightly and discovered this bug. *Issue* Our pipeline writes to a table with a decorator. When attempting to write to an existing partitioned table with a decorator, the write succeeds. When using a partitioned table destination that doesn't exist without a decorator, the write succeeds. *However, when writing to a partitioned table that doesn't exist with a decorator, the write fails*. *Example Implementation* {code:java} BigQueryIO.writeTableRows() .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) .withFailedInsertRetryPolicy(InsertRetryPolicy.alwaysRetry()) .to(new DynamicDestinations() { @Override public String getDestination(ValueInSingleWindow element) { return "PROJECT_ID:DATASET_ID.TABLE_ID$20170902"; } @Override public TableDestination getTable(String destination) { TimePartitioning DAY_PARTITION = new TimePartitioning().setType("DAY"); return new TableDestination(destination, null, DAY_PARTITION); } @Override public TableSchema getSchema(String destination) { return TABLE_SCHEMA; } }) {code} *Relevant Logs & Errors in StackDriver* {code:none} 23:06:26.790 Trying to create BigQuery table: PROJECT_ID:DATASET_ID.TABLE_ID$20170902 23:06:26.873 Invalid table ID \"TABLE_ID$20170902\". Table IDs must be alphanumeric (plus underscores) and must be at most 1024 characters long. Also, Table decorators cannot be used. {code} was: Dataflow Job ID: https://console.cloud.google.com/dataflow/job/2017-09-08_23_03_14-14637186041605198816?project=firebase-lessthan3 Tagging [~reuvenlax] as I believe he built the time partitioning integration that was merged into master. *Background* Our production pipeline ingests millions of events per day and routes events into our clients' numerous tables. To keep costs down, all of our tables are partitioned. However, this requires that we create the tables before we allow events to process as creating partitioned tables isn't supported in 2.1.0. We've been looking forward to [~reuvenlax]'s partition table write feature ([#3663|https://github.com/apache/beam/pull/3663]) to get merged into master for some time now as it'll allow us to launch our client platforms much, much faster. Today we got around to testing the 2.2.0 nightly and discovered this bug. *Issue* Our pipeline writes to a table with a decorator. When attempting to write to an existing partitioned table with a decorator, the write succeeds. When using a partitioned table destination that doesn't exist without a decorator, the write succeeds. *However, when writing to a partitioned table that doesn't exist with a decorator, the write fails*. *Example Implementation* {code:java} BigQueryIO.writeTableRows() .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) .withFailedInsertRetryPolicy(InsertRetryPolicy.alwaysRetry()) .to(new DynamicDestinations() { @Override public String getDestination(ValueInSingleWindow element) { return "PROJECT_ID:DATASET_ID.TABLE_ID$20170902"; } @Override public TableDestination getTable(String destination) { TimePartitioning DAY_PARTITION = new TimePartitioning().setType("DAY"); return new TableDestination(destination, null, DAY_PARTITION); } @Override public TableSchema getSchema(String destination) { return TABLE_SCHEMA; } }) {code} *Relevant Logs & Errors in StackDriver* {code:none} 23:06:26.790 Trying to create BigQuery table: PROJECT_ID:DATASET_ID.TABLE_ID$20170902 23:06:26.873 Invalid table ID \"TABLE_ID$20170902\". Table IDs must be alphanumeric (plus underscores) and must be at most 1024 characters long. Also, Table decorators cannot be used. {code} > BQ Partitioned Table Write Fails When Destination has Partition Decorator > ------------------------------------------------------------------------- > > Key: BEAM-2870 > URL: https://issues.apache.org/jira/browse/BEAM-2870 > Project: Beam > Issue Type: Bug > Components: runner-dataflow > Affects Versions: 2.2.0 > Environment: Dataflow Runner, Streaming, 10 x (n1-highmem-8 & 500gb SDD) > Reporter: Steven Jon Anderson > Assignee: Thomas Groh > Labels: bigquery, dataflow, google, google-cloud-bigquery, google-dataflow > Fix For: 2.2.0 > > > Dataflow Job ID: https://console.cloud.google.com/dataflow/job/2017-09-08_23_03_14-14637186041605198816 > Tagging [~reuvenlax] as I believe he built the time partitioning integration that was merged into master. > *Background* > Our production pipeline ingests millions of events per day and routes events into our clients' numerous tables. To keep costs down, all of our tables are partitioned. However, this requires that we create the tables before we allow events to process as creating partitioned tables isn't supported in 2.1.0. We've been looking forward to [~reuvenlax]'s partition table write feature ([#3663|https://github.com/apache/beam/pull/3663]) to get merged into master for some time now as it'll allow us to launch our client platforms much, much faster. Today we got around to testing the 2.2.0 nightly and discovered this bug. > *Issue* > Our pipeline writes to a table with a decorator. When attempting to write to an existing partitioned table with a decorator, the write succeeds. When using a partitioned table destination that doesn't exist without a decorator, the write succeeds. *However, when writing to a partitioned table that doesn't exist with a decorator, the write fails*. > *Example Implementation* > {code:java} > BigQueryIO.writeTableRows() > .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) > .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND) > .withFailedInsertRetryPolicy(InsertRetryPolicy.alwaysRetry()) > .to(new DynamicDestinations() { > @Override > public String getDestination(ValueInSingleWindow element) { > return "PROJECT_ID:DATASET_ID.TABLE_ID$20170902"; > } > @Override > public TableDestination getTable(String destination) { > TimePartitioning DAY_PARTITION = new TimePartitioning().setType("DAY"); > return new TableDestination(destination, null, DAY_PARTITION); > } > @Override > public TableSchema getSchema(String destination) { > return TABLE_SCHEMA; > } > }) > {code} > *Relevant Logs & Errors in StackDriver* > {code:none} > 23:06:26.790 > Trying to create BigQuery table: PROJECT_ID:DATASET_ID.TABLE_ID$20170902 > 23:06:26.873 > Invalid table ID \"TABLE_ID$20170902\". Table IDs must be alphanumeric (plus underscores) and must be at most 1024 characters long. Also, Table decorators cannot be used. > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)