You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/04/01 12:59:24 UTC

[GitHub] [flink] JingsongLi opened a new pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

JingsongLi opened a new pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602
 
 
   
   ## What is the purpose of the change
   
   Add row support for parquet writer.
   
   ## Brief change log
   
   - Add ParquetRowDataBuilder
   - Add ParquetRowDataWriter
   
   Timestamp support:
   - We only support INT96 bytes now, julianDay(4) + nanosOfDay(8). See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
   - Timezone: provide a utcTimestamp bool, true using utc, false using local time zone. 
   
   Decimal support: 
   - isLegacyFormat is true: use FIXED_LEN_BYTE_ARRAY to support all decimal, this is compatible with hive and impala.
   - isLegacyFormat is false: use int32/int64/FIXED_LEN_BYTE_ARRAY to support all kinds of precision decimal. This is compatible with spark.
   
   ## Verifying this change
   
   `ParquetRowDataWriterTest`
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (JavaDocs)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b28a0ffe70a414addafd0d3b481292a2831d4e10 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/157506831) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937) 
   * 3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b28a0ffe70a414addafd0d3b481292a2831d4e10 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/157506831) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607235924
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit b28a0ffe70a414addafd0d3b481292a2831d4e10 (Wed Apr 01 13:03:17 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] leonardBang commented on a change in pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
leonardBang commented on a change in pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#discussion_r402758054
 
 

 ##########
 File path: flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriterTest.java
 ##########
 @@ -0,0 +1,166 @@
+/*
+ * 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.formats.parquet.row;
+
+import org.apache.flink.api.common.serialization.BulkWriter;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.formats.parquet.ParquetWriterFactory;
+import org.apache.flink.formats.parquet.vector.ParquetColumnarRowSplitReader;
+import org.apache.flink.formats.parquet.vector.ParquetSplitReaderUtil;
+import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.DataFormatConverters;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.hadoop.ParquetOutputFormat;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.charset.StandardCharsets;
+import java.time.LocalDateTime;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.IntStream;
+
+/**
+ * Test for {@link ParquetRowDataBuilder} and {@link ParquetRowDataWriter}.
+ */
+public class ParquetRowDataWriterTest {
+
+	@ClassRule
+	public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
+
+	private static final RowType ROW_TYPE = RowType.of(
+			new VarCharType(VarCharType.MAX_LENGTH),
+			new VarBinaryType(VarBinaryType.MAX_LENGTH),
+			new BooleanType(),
+			new TinyIntType(),
+			new SmallIntType(),
+			new IntType(),
+			new BigIntType(),
+			new FloatType(),
+			new DoubleType(),
+			new TimestampType(9),
+			new DecimalType(5, 0),
+			new DecimalType(15, 0),
+			new DecimalType(20, 0));
+
+	@SuppressWarnings("unchecked")
+	private static final DataFormatConverters.DataFormatConverter<BaseRow, Row> CONVERTER =
+			DataFormatConverters.getConverterForDataType(
+					TypeConversions.fromLogicalToDataType(ROW_TYPE));
+
+	@Test
+	public void testTypes() throws IOException {
+		Configuration conf = new Configuration();
+		innerTest(conf, true);
+		innerTest(conf, false);
+	}
+
+	@Test
+	public void testCompression() throws IOException {
+		Configuration conf = new Configuration();
+		conf.set(ParquetOutputFormat.COMPRESSION, "GZIP");
+		innerTest(conf, true);
+		innerTest(conf, false);
+	}
+
+	private void innerTest(
+			Configuration conf,
+			boolean utcTimestamp) throws IOException {
+		Path path = new Path(TEMPORARY_FOLDER.newFolder().getPath(), UUID.randomUUID().toString());
+		int number = 1000;
+		List<Row> rows = new ArrayList<>(number);
+		for (int i = 0; i < number; i++) {
+			Integer v = i;
+			rows.add(Row.of(
+					String.valueOf(v),
+					String.valueOf(v).getBytes(StandardCharsets.UTF_8),
+					v % 2 == 0,
+					v.byteValue(),
+					v.shortValue(),
+					v,
+					v.longValue(),
+					v.floatValue(),
+					v.doubleValue(),
+					toDateTime(v),
+					BigDecimal.valueOf(v),
+					BigDecimal.valueOf(v),
+					BigDecimal.valueOf(v)));
+		}
+
+		ParquetWriterFactory<BaseRow> factory = ParquetRowDataBuilder.createWriterFactory(
+				ROW_TYPE, conf, utcTimestamp);
+		BulkWriter<BaseRow> writer = factory.create(path.getFileSystem().create(
+				path, FileSystem.WriteMode.OVERWRITE));
+		for (int i = 0; i < number; i++) {
+			writer.addElement(CONVERTER.toInternal(rows.get(i)));
+		}
+		writer.flush();
+		writer.finish();
+
+		// verify
+		ParquetColumnarRowSplitReader reader = ParquetSplitReaderUtil.genPartColumnarRowReader(
+				utcTimestamp,
+				conf,
+				ROW_TYPE.getFieldNames().toArray(new String[0]),
+				ROW_TYPE.getChildren().stream()
+						.map(TypeConversions::fromLogicalToDataType)
+						.toArray(DataType[]::new),
+				new HashMap<>(),
+				IntStream.range(0, ROW_TYPE.getFieldCount()).toArray(),
+				50,
+				path,
+				0,
+				Long.MAX_VALUE);
+		int cnt = 0;
+		while (!reader.reachedEnd()) {
+			Row row = CONVERTER.toExternal(reader.nextRecord());
+			Assert.assertEquals(rows.get(cnt), row);
+			cnt++;
+		}
+		Assert.assertEquals(number, cnt);
+	}
+
+	private LocalDateTime toDateTime(Integer v) {
+		v = (v > 0 ? v : -v) % 10000;
 
 Review comment:
   the `10000` looks like a wrong digit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 95f0deb44dfae1be800260732e47bf8ceaf9ef67 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/157722695) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976) 
   * 5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/158122639) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b28a0ffe70a414addafd0d3b481292a2831d4e10 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/157506831) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158143207",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7020",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/158122639) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010) 
   * 39d9b1a9c461fc940a660b95a4764f9982ba1599 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/158143207) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7020) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/158122639) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158143207",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7020",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 39d9b1a9c461fc940a660b95a4764f9982ba1599 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/158143207) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7020) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] JingsongLi merged pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
JingsongLi merged pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/157712101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968) 
   * 95f0deb44dfae1be800260732e47bf8ceaf9ef67 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b28a0ffe70a414addafd0d3b481292a2831d4e10 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/157506831) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937) 
   * 3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/157712101) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] leonardBang commented on a change in pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
leonardBang commented on a change in pull request #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#discussion_r402756699
 
 

 ##########
 File path: flink-formats/flink-parquet/src/main/java/org/apache/flink/formats/parquet/row/ParquetRowDataWriter.java
 ##########
 @@ -0,0 +1,325 @@
+/*
+ * 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.formats.parquet.row;
+
+import org.apache.flink.table.dataformat.BaseRow;
+import org.apache.flink.table.dataformat.Decimal;
+import org.apache.flink.table.dataformat.SqlTimestamp;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.Type;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.sql.Timestamp;
+import java.util.Arrays;
+
+import static org.apache.flink.formats.parquet.utils.ParquetSchemaConverter.computeMinBytesForDecimalPrecision;
+import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.JULIAN_EPOCH_OFFSET_DAYS;
+import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.MILLIS_IN_DAY;
+import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_MILLISECOND;
+import static org.apache.flink.formats.parquet.vector.reader.TimestampColumnReader.NANOS_PER_SECOND;
+
+/**
+ * Writes a record to the Parquet API with the expected schema in order to be written to a file.
+ */
+public class ParquetRowDataWriter {
+
+	private final RecordConsumer recordConsumer;
+	private final boolean utcTimestamp;
+
+	private final FieldWriter[] filedWriters;
+	private final String[] fieldNames;
+
+	public ParquetRowDataWriter(
+			RecordConsumer recordConsumer,
+			RowType rowType,
+			GroupType schema,
+			boolean utcTimestamp) {
+		this.recordConsumer = recordConsumer;
+		this.utcTimestamp = utcTimestamp;
+
+		this.filedWriters = new FieldWriter[rowType.getFieldCount()];
+		this.fieldNames = rowType.getFieldNames().toArray(new String[0]);
+		for (int i = 0; i < rowType.getFieldCount(); i++) {
+			this.filedWriters[i] = createWriter(rowType.getTypeAt(i), schema.getType(i));
+		}
+	}
+
+	/**
+	 * It writes a record to Parquet.
+	 *
+	 * @param record Contains the record that is going to be written.
+	 */
+	public void write(final BaseRow record) {
+		recordConsumer.startMessage();
+		for (int i = 0; i < filedWriters.length; i++) {
+			if (!record.isNullAt(i)) {
+				String fieldName = fieldNames[i];
+				FieldWriter writer = filedWriters[i];
+
+				recordConsumer.startField(fieldName, i);
+				writer.write(record, i);
+				recordConsumer.endField(fieldName, i);
+			}
+		}
+		recordConsumer.endMessage();
+	}
+
+	private FieldWriter createWriter(LogicalType t, Type type) {
+		if (type.isPrimitive()) {
+			switch (t.getTypeRoot()) {
+				case CHAR:
+				case VARCHAR:
+					return new StringWriter();
+				case BOOLEAN:
+					return new BooleanWriter();
+				case BINARY:
+				case VARBINARY:
+					return new BinaryWriter();
+				case DECIMAL:
+					DecimalType decimalType = (DecimalType) t;
+					return makeDecimalWriter(decimalType.getPrecision(), decimalType.getScale());
+				case TINYINT:
+					return new ByteWriter();
+				case SMALLINT:
+					return new ShortWriter();
+				case DATE:
+				case TIME_WITHOUT_TIME_ZONE:
+				case INTEGER:
+					return new IntWriter();
+				case BIGINT:
+					return new LongWriter();
+				case FLOAT:
+					return new FloatWriter();
+				case DOUBLE:
+					return new DoubleWriter();
+				case TIMESTAMP_WITHOUT_TIME_ZONE:
+					TimestampType timestampType = (TimestampType) t;
+					return new TimestampWriter(timestampType.getPrecision());
+				case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+					LocalZonedTimestampType localZonedTimestampType = (LocalZonedTimestampType) t;
+					return new TimestampWriter(localZonedTimestampType.getPrecision());
+				default:
+					throw new UnsupportedOperationException("Unsupported type: " + type);
+			}
+		} else {
+			throw new IllegalArgumentException("Unsupported  data type: " + t);
+		}
+	}
+
+	private interface FieldWriter {
+
+		void write(BaseRow row, int ordinal);
+	}
+
+	private class BooleanWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addBoolean(row.getBoolean(ordinal));
+		}
+	}
+
+	private class ByteWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addInteger(row.getByte(ordinal));
+		}
+	}
+
+	private class ShortWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addInteger(row.getShort(ordinal));
+		}
+	}
+
+	private class LongWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addLong(row.getLong(ordinal));
+		}
+	}
+
+	private class FloatWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addFloat(row.getFloat(ordinal));
+		}
+	}
+
+	private class DoubleWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addDouble(row.getDouble(ordinal));
+		}
+	}
+
+	private class StringWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addBinary(
+				Binary.fromReusedByteArray(row.getString(ordinal).getBytes()));
+		}
+	}
+
+	private class BinaryWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addBinary(
+					Binary.fromReusedByteArray(row.getBinary(ordinal)));
+		}
+	}
+
+	private class IntWriter implements FieldWriter {
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addInteger(row.getInt(ordinal));
+		}
+	}
+
+	/**
+	 * We only support INT96 bytes now, julianDay(4) + nanosOfDay(8).
+	 * See https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#timestamp
+	 * TIMESTAMP_MILLIS and TIMESTAMP_MICROS are the deprecated ConvertedType.
+	 */
+	private class TimestampWriter implements FieldWriter {
+
+		private final int precision;
+
+		private TimestampWriter(int precision) {
+			this.precision = precision;
+		}
+
+		@Override
+		public void write(BaseRow row, int ordinal) {
+			recordConsumer.addBinary(timestampToInt96(row.getTimestamp(ordinal, precision)));
+		}
+	}
+
+	private Binary timestampToInt96(SqlTimestamp sqlTimestamp) {
+		int julianDay;
+		long nanosOfDay;
+		if (utcTimestamp) {
+			long mills = sqlTimestamp.getMillisecond();
+			julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS);
+			nanosOfDay = (mills % MILLIS_IN_DAY) * NANOS_PER_MILLISECOND + sqlTimestamp.getNanoOfMillisecond();
+		} else {
+			Timestamp timestamp = sqlTimestamp.toTimestamp();
+			long mills = timestamp.getTime();
+			julianDay = (int) ((mills / MILLIS_IN_DAY) + JULIAN_EPOCH_OFFSET_DAYS);
+			nanosOfDay = ((mills % MILLIS_IN_DAY) / 1000) * NANOS_PER_SECOND + timestamp.getNanos();
+		}
+
+		ByteBuffer buf = ByteBuffer.allocate(12);
+		buf.order(ByteOrder.LITTLE_ENDIAN);
+		buf.putLong(nanosOfDay);
+		buf.putInt(julianDay);
+		buf.flip();
+		return Binary.fromConstantByteBuffer(buf);
+	}
+
+	private FieldWriter makeDecimalWriter(int precision, int scale) {
 
 Review comment:
   I think `#createDecimalWriter` is better to align `#createWriter`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/158122639) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot commented on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b28a0ffe70a414addafd0d3b481292a2831d4e10 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158143207",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7020",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 39d9b1a9c461fc940a660b95a4764f9982ba1599 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/158143207) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7020) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b28a0ffe70a414addafd0d3b481292a2831d4e10 Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/157506831) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 95f0deb44dfae1be800260732e47bf8ceaf9ef67 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/157722695) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "FAILURE",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 95f0deb44dfae1be800260732e47bf8ceaf9ef67 Travis: [FAILURE](https://travis-ci.com/github/flink-ci/flink/builds/157722695) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976) 
   * 5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "SUCCESS",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/158122639",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010",
       "triggerID" : "5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe",
       "triggerType" : "PUSH"
     }, {
       "hash" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "39d9b1a9c461fc940a660b95a4764f9982ba1599",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5dce0e57680a2e3b34f27ec2fac8ed7a59d00dbe Travis: [SUCCESS](https://travis-ci.com/github/flink-ci/flink/builds/158122639) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=7010) 
   * 39d9b1a9c461fc940a660b95a4764f9982ba1599 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [flink] flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #11602: [FLINK-16912][parquet] Introduce table row write support for parquet writer
URL: https://github.com/apache/flink/pull/11602#issuecomment-607241516
 
 
   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157506831",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6937",
       "triggerID" : "b28a0ffe70a414addafd0d3b481292a2831d4e10",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "CANCELED",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157712101",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968",
       "triggerID" : "3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "PENDING",
       "url" : "https://travis-ci.com/github/flink-ci/flink/builds/157722695",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     }, {
       "hash" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976",
       "triggerID" : "95f0deb44dfae1be800260732e47bf8ceaf9ef67",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 3f7c4f6dd3c6f9cdb66d9d5a1dcb456b567436c3 Travis: [CANCELED](https://travis-ci.com/github/flink-ci/flink/builds/157712101) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6968) 
   * 95f0deb44dfae1be800260732e47bf8ceaf9ef67 Travis: [PENDING](https://travis-ci.com/github/flink-ci/flink/builds/157722695) Azure: [PENDING](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=6976) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services