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/12/14 07:41:50 UTC

[GitHub] [flink] maosuhan opened a new pull request #14376: [FLINK-18202][PB format] New Format of protobuf

maosuhan opened a new pull request #14376:
URL: https://github.com/apache/flink/pull/14376


    ## What is the purpose of the change
   
   Protobuf is a structured data format introduced by google. Compared to json, protobuf is more efficient of space and computing. Nowadays, many companies use protobuf instead of json as data format in kafka and streaming processing.
   
   So, we will introduce a new format which can derialize/serialize protobuf data in a fast speed.
   User can use this format in SQL or Table API. 
   
   
   ## Verifying this change
   
   `
   create table source(
   ....... column list
   ) 
                   with(
                   'connector' = 'kafka',
                   'format' = 'pb',
                   'pb.message-class-name' = '<message class name>'
   )
   `
   
   `
   create table sink(
   ....... column list
   ) 
   with(
                   'connector' = 'kafka',
                   'format' = 'pb',
                   'pb.message-class-name' = '<message class name>'
   )
   `
   
   ##Tests
   TODO
   
   ## Benchmark
   Performance test for pb object containing 200+ fields. Below is the consumed time of processing 10M rows.
   Implementation | Deserialize Speed | Serialize Speed
   -- | -- | --
   json | 110s | 120s
   DynamicMessage and Descriptor API | 152s | 206s
   Codegen | 42s | 33s
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - New dependencies: Add protobuf dependency com.google.protobuf:protobuf-java:3.12.2
     - Public API: Add new format in Flink SQL
     - The serializers: Add new PbRowDeserializationSchema and PbRowSerializationSchema
     - The runtime per-record code paths (performance sensitive): yes
   
   ## Documentation
   Connector params:
   
   1. pb.message-class-name: Required option to specify the full name of protobuf message class. The protobuf class 
   must be located in the classpath both in client and task side.
   1. pb.read-default-values: Optional flag to read as default values instead of null when some field does not exist in deserialization; default to false. If proto syntax is proto3, this value will be set true forcibly because proto3's standard is to use default values.
   1. pb.ignore-parse-errors: default is false. Deserialization task will keep running if pb parse error occurs.
   
   ## Limitation
   1. In proto definition, package must be equals to java_package.
   2. java_multiple_files must be true
   
   ## Notice
   
   ### default values
   As you know, if the syntax is proto2, the generated pb class has bit flags to indicate whether a field is set or not. We can use pbObject.hasXXX() method to know whether the field is set or not. In this way, we can handle null information in flink properly. So if syntax=2,the decoded flink row may contain null values. We could also expose an option to user to control the behavior to handle null values.
   But if the syntax is proto3, the generated pb class does not have pbObject.hasXXX() method and does not hold bit flags, so there is no way to tell if a field is set or not if it is equals to default value. For example, if pbObje.getDim1() returns 0, there's no way to tell if dim1 is set by 0 or it is not set anyway. So if syntax=3, the decoded flink row will not contain any null values.
    
   Also pb does not permit null in key/value of map and array. We need to generate default value for them.
   
   row value | pb value
   -- | --
   map<string,string>(<"a", null>) | map<string,string>(("a", ""))
   map<string,string>(<null, "a">) | map<string,string>(("", "a"))
   map<int, int>(null, 1) | map<int, int>(0, 1)
   map<int, int>(1, null) | map<int, int>(1, 0)
   map<long, long>(null, 1) | map<long, long>(0, 1)
   map<long, long>(1, null) | map<long, long>(1, 0)
   map<bool, bool>(null, true) | map<bool, bool>(false, true)
   map<bool, bool>(true, null) | map<bool, bool>(true, false)
   map<string, float>("key", null) | map<string, float>("key", 0)
   map<string, double>("key", null) | map<string, double>("key", 0)
   map<string, enum>("key", null) | map<string, enum>("key", first_enum_element)
   map<string, binary>("key", null) | map<string, binary>("key", ByteString.EMPTY)
   map<string, MESSAGE>("key", null) | map<string, MESSAGE>("key", MESSAGE.getDefaultInstance())
   array<string>(null) | array("")
   array<int>(null) | array(0)
   array<long>(null) | array(0)
   array<bool>(null) | array(false)
   array<float>(null) | array(0)
   array<double>(null) | array(0)
   array<enum>(null) | array(first_enum_element)
   array<binary>(null) | array(ByteString.EMPTY)
   array<message>(null) | array(MESSAGE.getDefaultInstance())
   
   ### OneOf field
   In serialization process, there's no guarantee that the flink row fields of one-of group only contains at least one non-null value.
   So in serialization, we set each field in the order of flink schema, so the field in high position will override then field of low position in the same one-of group.
   
   ### Enum type
   Enum value of pb will be converted to String and vice versa in the name of enum value definition in pb. 
   


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r553095717



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()
+			.equals(descriptor.getFile().getPackage())) {
+			throw new IllegalArgumentException(
+				"java_package and package must be the same in proto definition");
+		}
+		if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       If someone can tell me how to get the real java class name from com.google.protobuf.Descriptors.Descriptor, the problem can be solved.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 248aaeec8b1c42ca82610eebe34e2321d95f0320 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043) 
   
   <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



[GitHub] [flink] laughingman7743 commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r649871292



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    // protobuf code has a bug that, f_abc_7d will be convert to fAbc7d, but actually we need fAbc7D
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                // This closely matches the logic for ASCII characters in:
+                // http://google3/google/protobuf/descriptor.cc?l=249-251&rcl=228891689
+                if ('a' <= ch && ch <= 'z') {
+                    ch = (char) (ch - 'a' + 'A');
+                    isNextUpperCase = false;
+                }
+                result.append(ch);
+            } else {
+                result.append(ch);
+            }
+        }
+        return result.toString();
+    }
+
+    public static boolean isSimpleType(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case BOOLEAN:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+            case BINARY:
+            case VARBINARY:
+                return true;
+            default:
+                return false;
+        }
+    }
+
+    public static String getStrongCamelCaseJsonName(String name) {
+        String jsonName = fieldNameToJsonName(name);
+        if (jsonName.length() == 1) {
+            return jsonName.toUpperCase();
+        } else {
+            return jsonName.substring(0, 1).toUpperCase() + jsonName.substring(1);
+        }
+    }
+
+    public static Descriptors.Descriptor getDescriptor(String className) {
+        try {
+            Class<?> pbClass = Class.forName(className);

Review comment:
       We may need to take the UserCodeClassLoader as an argument and specify the class loader in the Class.forName call, as shown below.
   ```
   public static Descriptors.Descriptor getDescriptor(ClassLoader classLoader, String className) {
       try {
           Class<?> pbClass = Class.forName(className, false, classLoader);
   ```




-- 
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



[GitHub] [flink] syucream commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
syucream commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r547024123



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()

Review comment:
       Why does it check such the condition? According to [the spec of protobuf](https://developers.google.com/protocol-buffers/docs/javatutorial#defining-your-protocol-format), we can omit `java_package` or set a different value with `package`.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * e3f953d91778ef96ad5fe222dd9da229dc2c5619 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * d7b612acb72c25b4a30899155b764b6ad2535613 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612) 
   * 248aaeec8b1c42ca82610eebe34e2321d95f0320 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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-814578490


   @sv3ndk 
   Thanks for your detailed explanation of the issue.
   I checked the MR you raised, it seems you ignore the case of protobuf version 2. In protobuf 2, user can use `hasXXX` method to tell if a scalar field is null or not. In our company, we use protobuf 2 a lot because we can use this function.
   What do you think about it?


-- 
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



[GitHub] [flink] MartijnVisser edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1006373934


   @maosuhan Because the release 1.15 cut is coming up quickly (beginning of February), most of the committers are working to finish up their committed stuff. I'm not sure it can be achieved before that time, but I'm trying to find some who can look at it. Sorry for the delay :(


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * ccf96fb549f1917fe888d69466a7e10013aa76ca Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717) 
   * 12bc9ca2640c7773b2ca7fa50e204605183ff309 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * ccf96fb549f1917fe888d69466a7e10013aa76ca Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717) 
   * 12bc9ca2640c7773b2ca7fa50e204605183ff309 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhuzhengjun01 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
zhuzhengjun01 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1016191275


   > 
   like this , 
   `syntax = "proto3";
   package org.apache.flink.formats.protobuf.proto;
   option java_multiple_files = false;
   
   message KafkaSinkTestTT {
       oneof name_oneof {
           string name = 1;
   
       }
       oneof num_oneof {
           int32 num = 2;
       }
       oneof tt_oneof {
           int64 xtime = 3;
   
       }
       ro row1 = 4;
       message ro{
           string a = 1;
           int32 b = 2;
       }
   }
   `
   actually, I want to distinguish between default and missing values in proto3
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r597477533



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenRowDeserializer.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+
+/** Deserializer to convert proto message type object to flink row type data. */
+public class PbCodegenRowDeserializer implements PbCodegenDeserializer {
+    private Descriptor descriptor;
+    private RowType rowType;
+    private boolean readDefaultValues;
+    private PbCodegenAppender appender = new PbCodegenAppender();
+
+    public PbCodegenRowDeserializer(
+            Descriptor descriptor, RowType rowType, boolean readDefaultValues) {
+        this.rowType = rowType;
+        this.descriptor = descriptor;
+        this.readDefaultValues = readDefaultValues;
+    }
+
+    @Override
+    public String codegen(String returnInternalDataVarName, String pbGetStr)
+            throws PbCodegenException {
+        // The type of messageGetStr is a native pb object,
+        // it should be converted to RowData of flink internal type
+        PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+        int uid = varUid.getAndIncrement();
+        String pbMessageVar = "message" + uid;
+        String rowDataVar = "rowData" + uid;
+
+        int fieldSize = rowType.getFieldNames().size();
+        String pbMessageTypeStr = PbFormatUtils.getFullJavaName(descriptor);
+        appender.appendLine(pbMessageTypeStr + " " + pbMessageVar + " = " + pbGetStr);
+        appender.appendLine(
+                "GenericRowData " + rowDataVar + " = new GenericRowData(" + fieldSize + ")");
+        int index = 0;
+        for (String fieldName : rowType.getFieldNames()) {
+            int subUid = varUid.getAndIncrement();
+            String elementDataVar = "elementDataVar" + subUid;
+
+            LogicalType subType = rowType.getTypeAt(rowType.getFieldIndex(fieldName));
+            FieldDescriptor elementFd = descriptor.findFieldByName(fieldName);
+            String strongCamelFieldName = PbFormatUtils.getStrongCamelCaseJsonName(fieldName);
+            PbCodegenDeserializer codegen =
+                    PbCodegenDeserializeFactory.getPbCodegenDes(
+                            elementFd, subType, readDefaultValues);
+            appender.appendLine("Object " + elementDataVar + " = null");
+            if (!readDefaultValues) {
+                // only works in syntax=proto2 and readDefaultValues=false
+                // readDefaultValues must be true in pb3 mode
+                String isMessageNonEmptyStr =
+                        isMessageNonEmptyStr(pbMessageVar, strongCamelFieldName, elementFd);
+                appender.appendSegment("if(" + isMessageNonEmptyStr + "){");
+            }
+            String elementMessageGetStr =
+                    pbMessageElementGetStr(pbMessageVar, strongCamelFieldName, elementFd);
+            if (!elementFd.isRepeated()) {
+                // field is not map or array
+                // this step is needed to convert primitive type to boxed type to unify the object

Review comment:
       yes, you are right!




-- 
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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r553102794



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()
+			.equals(descriptor.getFile().getPackage())) {
+			throw new IllegalArgumentException(
+				"java_package and package must be the same in proto definition");
+		}
+		if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       The complicated code is below, you can see that "OuterClass" is hard coded.
   ```
       public static String getJavaFullName(Descriptors.Descriptor descriptor) {
           String javaPackageName = descriptor.getFile().getOptions().getJavaPackage();
           if (descriptor.getFile().getOptions().getJavaMultipleFiles()) {
               //multiple_files=true
               if (null != descriptor.getContainingType()) {
                   //nested type
                   String parentJavaFullName = getJavaFullName(descriptor.getContainingType());
                   return parentJavaFullName + "." + descriptor.getName();
               } else {
                   //top level message
                   return javaPackageName + "." + descriptor.getName();
               }
           } else {
               //multiple_files=false
               if (null != descriptor.getContainingType()) {
                   //nested type
                   String parentJavaFullName = getJavaFullName(descriptor.getContainingType());
                   return parentJavaFullName + "." + descriptor.getName();
               } else {
                   //top level message
                   if (!descriptor.getFile().getOptions().hasJavaOuterClassname()) {
                       //user do not define outer class name in proto file
                       return javaPackageName + "." + descriptor.getName() + "OuterClass" + "." + descriptor.getName();
                   } else {
                       String outerName = descriptor.getFile().getOptions().getJavaOuterClassname();
                       //user define outer class name in proto file
                       return javaPackageName + "." + outerName + "." + descriptor.getName();
                   }
               }
           }
       }
   ```




----------------------------------------------------------------
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



[GitHub] [flink] ruanhang1993 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-786451333


   HI, @maosuhan thanks a lot for your work. I have used the code based on flink 1.11.2. There are some problem for me when using the protobuf format:
   
   1. protobuf-java jar in flink-protobuf conflicts with which in flink-dist and flink-sql-connector-hive
   2. There are some problems using protobuf format when recovering from some checkpoint
   ```
   Caused by: java.io.IOException: Failed to deserialize PB object.
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:97) ~[xxx.jar:?]
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:39) ~[xxx.jar:?]
   	at org.apache.flink.api.common.serialization.DeserializationSchema.deserialize(DeserializationSchema.java:81) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper.deserialize(KafkaDeserializationSchemaWrapper.java:56) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.partitionConsumerRecordsHandler(KafkaFetcher.java:181) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.runFetchLoop(KafkaFetcher.java:141) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755) ~[xxx.jar:?]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:213) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   Caused by: java.lang.NullPointerException
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:91) ~[xxx.jar:?]
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:39) ~[xxx.jar:?]
   	at org.apache.flink.api.common.serialization.DeserializationSchema.deserialize(DeserializationSchema.java:81) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper.deserialize(KafkaDeserializationSchemaWrapper.java:56) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.partitionConsumerRecordsHandler(KafkaFetcher.java:181) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.runFetchLoop(KafkaFetcher.java:141) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755) ~[xxx.jar:?]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:213) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   ```
   
   For problem 1,I resolve it by using the protobuf-java jar in flink-dist,and set the scope of protobuf-java in flink-protobuf to provided. Besides, I shaded the protobuf-java in flink-sql-connector-hive.
   ```
   // flink-protobuf : pom.xml
   <dependency>
   	<groupId>com.google.protobuf</groupId>
   	<artifactId>protobuf-java</artifactId>
   	<version>${protoc.version}</version>
   	<scope>provided</scope>
   </dependency>
   
   // flink-sql-connector-hive-1.2.2 :pom.xml :shade-plugin
   <relocations>
   	<relocation>
   		<pattern>com.google.protobuf</pattern>
   		<shadedPattern>org.apache.flink.connector.sql.hive.com.google.protobuf</shadedPattern>
   	</relocation>
   </relocations>
   ```
   For problem 2 (when submit job by `flink run -s <checkpoint>`), I simply used the following code for `deserialize` method in `PbRowDeserializationSchema` class to fix this issue.
   ```java
   @Override
   public RowData deserialize(byte[] message) throws IOException {
       try {
           if(protoToRowConverter == null) {
               protoToRowConverter = new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
   	}
           return protoToRowConverter.convertProtoBinaryToRow(message);
       } catch (Throwable t) {
               if (ignoreParseErrors) {
                   return null;
               }
               LOG.error("Failed to deserialize PB object.", t);
               throw new IOException("Failed to deserialize PB object.", t);
       }
   }
   ```
   
   I don't know if the question is that the way i use or the different flink version 1.11.2, hope the problems for me are helpful for this PR.
   Do you have any better advice for me to fix it ? 
   
   Thanks for Reading.
   


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-828337379


   @wuchong @libenchao 
   I have introduced a new connector param `write-null-string-literal`
   ```
   When serializing to protobuf data, this is the optional config to specify the string literal in protobuf's array/map in case of null values. By default empty string is used.
   ```
   ## Extra improvements
   1. I use `SimpleCompiler` of janino to compile all my code which is the same way in `org.apache.flink.table.runtime.generated.CompileUtils`
   1. Add IT cases of source and sink in SQL.
   1. Add `write-null-string-literal`.


-- 
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



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1006373934


   @maosuhan Because the release 1.15 cut is coming up quickly (beginning of February), most of the committers are working to finish up their committed stuff. I'm not sure it can be achieved before that time, but I'm trying to find some who can look at it. 


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-971310164


   @complone thanks for your attention and suggestion. I think we have already discussed a lot in the thread and this MR is pretty much ready for merge. 
   Actually the reason for this MR pending for a long time is that I do not know what I should do next to push this MR to the main ranch. Do you have any comment on current protobuf implementation and maybe we can work together.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * ccf96fb549f1917fe888d69466a7e10013aa76ca Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717) 
   * 12bc9ca2640c7773b2ca7fa50e204605183ff309 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * ccf96fb549f1917fe888d69466a7e10013aa76ca Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-988473669


   @MyLanPangzi I have updated the code to adaptively convert between enum of protobuf and numeric/string type of flink and also unit tests are added.
   Since codegen in java is a little tricky and hard to read, I have converted source files containing codegen logic to scala files.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-802704116


   @libenchao Hi Benchao, very thanks for your review effort and it is a great appreciate that you give many detailed suggestions.
   I have fixed most of the issues and there's only 3 issues remaining that we should discuss.
   
   1. should we use the user classloader to init protobuf class?
   2. if flink row contains null value in complex type, shall we offer user a parameter to control whether the task should raise exception or fill in default values?  I'm okay with that.
   3. Regarding codegen framework, I'm trying to follow the way that flink itself does. Flink just concat piece of java codes together and I don't know if there's a better way to do this. I guess the main reason why the readability is not good is that java does not have text block function like scala has.


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 248aaeec8b1c42ca82610eebe34e2321d95f0320 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043) 
   * bd27dabd13c11fd65b98423595e9d93234538444 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



[GitHub] [flink] sv3ndk edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
sv3ndk edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-809618142


   Thanks a lot for working on a Protobuf format! We also use that serialization mechanism at my company and will need such format  soon, this is very useful.
   
   I have a different understanding regarding the handling of missing values by protobuf 3 and I believe we should be able to provide Fllink with nulls in case of missing value in pb. @maosuhan, if you want I'd be happy to collaborate with you on this and add support for this case.
   
   As I understand it:
   
   * protobuf 3 considers every field as optional
   * if a pb field is a complex type, the generated java code contains a `hasXYZ()` method to determine if that field is present
   * if a pb field is a scalar, no such method gets generated
   * when a pb field value is not specified in a pb instance, the `getXYZ()` method returns the default value (i.e `""` instead of `null` in case of string)
   
   The way we approach this where I work is:
   * we only only use protobuf scalar types for non nullable fields
   * we rely on wrappers (like `google.protobuf.StringValue`) for nullable java primitives
   * when reading data, we always check `hasXYZ()` before calling `getXYZ()`
   
   Here's a quick example, generating java classes with pb `3.15.0`:
   
   Given this schema:
   ```
   syntax = "proto3";
   import "google/protobuf/wrappers.proto";
   
   message Parent {
     string id = 1;
     google.protobuf.StringValue name = 2;
     Inner innerValue = 3;
   
     message Inner {
       string f1 = 1;
       string f2 = 2;
     }
   }
   ```
   
   and this `roundTrip()` method:
   
   ```java
     Parent roundTrip(Parent parent) throws InvalidProtocolBufferException {
       return Parent.parser().parseFrom(parent.toByteArray());
     }
   ```
   
   Those assertions show that the missing `name` field can be correctly interpreted both before and after serialization:
   
   ```java
    var withoutName = Parent.newBuilder()
           .setId("theId")
           .setInnerValue(
               Parent.Inner.newBuilder()
                   .setF1("theF1")
                   .setF2("theF1")
                   .build()
           )
           .build();
   
       assert ! withoutName.hasName();
       assert ! roundTrip(withoutName).hasName();
       assert withoutName.hasInnerValue();
       assert roundTrip(withoutName).hasInnerValue();
       assert withoutName.getInnerValue().getF1().equals("theF1");
       assert roundTrip(withoutName).getInnerValue().getF1().equals("theF1");
       //assert ! hasNoName.hasId();           // does not compile: hasXYZ() does not exist if XYZ is a scalar
   ```
   
   Similarly, this instance with a missing nested field can be interpreted correctly as well by a reader:
   
   ```java
    var withoutInner = Parent.newBuilder()
           .setId("theId")
           .setName(StringValue.newBuilder().setValue("theName").build())
           .build();
   
       assert ! withoutInner.hasInnerValue();
       assert ! roundTrip(withoutInner).hasInnerValue();
       assert withoutInner.hasName();
       assert roundTrip(withoutInner).hasName();
       assert withoutInner.getName().getValue().equals("theName");
       assert roundTrip(withoutInner).getName().getValue().equals("theName");
   ```
   


-- 
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



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-976380241


   @maosuhan Sounds good to me. One question from my end, does this PR allow you to use Protobuf both in the DataStream and the Table API/SQL?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r767101531



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.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.protobuf;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.google.protobuf.Descriptors;
+import org.apache.commons.lang3.StringUtils;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    /**
+     * protobuf code has a bug that, f_abc_7d will be convert to fAbc7d in {@code
+     * com.google.protobuf.Descriptors.FileDescriptor.getJsonName()}, but actually we need fAbc7D.
+     */
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                if ('a' <= ch && ch <= 'z') {
+                    ch = (char) (ch - 'a' + 'A');
+                    isNextUpperCase = false;
+                }
+                result.append(ch);
+            } else {
+                result.append(ch);
+            }
+        }
+        return result.toString();
+    }
+
+    private static String getJavaPackageFromProtoFile(Descriptors.Descriptor descriptor) {
+        boolean hasJavaPackage = descriptor.getFile().getOptions().hasJavaPackage();
+        if (hasJavaPackage) {
+            String javaPackage = descriptor.getFile().getOptions().getJavaPackage();
+            if (StringUtils.isBlank(javaPackage)) {
+                throw new FlinkRuntimeException("java_package cannot be blank string");
+            }
+            return javaPackage;
+        } else {
+            String packageName = descriptor.getFile().getPackage();
+            if (StringUtils.isBlank(packageName)) {
+                throw new FlinkRuntimeException("package and java_package cannot both be empty");
+            }
+            return packageName;
+        }
+    }
+
+    public static String getFullJavaName(Descriptors.Descriptor descriptor) {
+        String javaPackageName = getJavaPackageFromProtoFile(descriptor);
+        if (descriptor.getFile().getOptions().getJavaMultipleFiles()) {
+            // multiple_files=true
+            if (null != descriptor.getContainingType()) {
+                // nested type
+                String parentJavaFullName = getFullJavaName(descriptor.getContainingType());
+                return parentJavaFullName + "." + descriptor.getName();
+            } else {
+                // top level message
+                return javaPackageName + "." + descriptor.getName();
+            }
+        } else {
+            // multiple_files=false
+            if (null != descriptor.getContainingType()) {
+                // nested type
+                String parentJavaFullName = getFullJavaName(descriptor.getContainingType());
+                return parentJavaFullName + "." + descriptor.getName();
+            } else {
+                // top level message
+                if (!descriptor.getFile().getOptions().hasJavaOuterClassname()) {
+                    // user do not define outer class name in proto file
+                    return javaPackageName

Review comment:
       @MyLanPangzi Thanks for your findings. Yes, it is a bug. I found that if the proto file name is test.proto and the content is as below, the outermost class name will be TestOuterClass because the message name is the same with file name.
   ```
   syntax = "proto2";
   message Test {
   ....
   }
   ```
   
   It is a little tricky to get java full class name from the `Descriptors.Descriptor`, so I use a simpler way that I extract the outer prefix name from `protobuf.message-class-name`.
   ```
       public static String getOuterProtoPrefix(String name) {
           name = name.replace('$', '.');
           int index = name.lastIndexOf('.');
           if (index != -1) {
               // include dot
               return name.substring(0, index + 1);
           } else {
               return "";
           }
       }
   ``` 
   I have fixed that issue and added additional unit tests.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] complone commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
complone commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-967745835


   > FLINK-18202
   @MartijnVisser 
   Yes, I want to continue working for FLINK-18202


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MyLanPangzi commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MyLanPangzi commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-982246921


   @maosuhan Can we support enum type return Int ? Thanks.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-972994248


    @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-975382299


   @MartijnVisser Much thanks for your support and suggestions. I have rebase my commits and make them cleaner to review. And I also make this PR pass the azure IC. 
   Could we begin the code review process?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhuzhengjun01 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
zhuzhengjun01 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1016191275


   > 
   like this , 
   `syntax = 'proto3';
   package org.apache.flink.formats.protobuf.proto;
   option java_multiple_files = false;
   
   message KafkaSinkTestTT {
       oneof name_oneof {
           string name = 1;
   
       }
       oneof num_oneof {
           int32 num = 2;
       }
       oneof tt_oneof {
           int64 xtime = 3;
   
       }
       ro row1 = 4;
       message ro{
           string a = 1;
           int32 b = 2;
       }
   }
   `
   actually, I want to distinguish between default and missing values in proto3
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhuzhengjun01 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
zhuzhengjun01 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1016148079


   @maosuhan hi, flink-protobuf works very well in my job, but i found some corner case that it was not supported `oneof` in proto3.  so this will be dealt in future or not ?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] libenchao commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
libenchao commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-794763304


   @maosuhan I just pulled your code into my local repo and there are many check-style violations, could you resolve these problems first and make the CI passed?


----------------------------------------------------------------
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



[GitHub] [flink] wuchong commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
wuchong commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-754386961


   Hi @maosuhan , the community changed the code format recently. Could you rebase your branch to the lastest master and update the format according to this doc? https://ci.apache.org/projects/flink/flink-docs-master/flinkDev/ide_setup.html#code-formatting
   
   cc @libenchao , do you have time to review this PR?


----------------------------------------------------------------
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



[GitHub] [flink] ruanhang1993 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-790247456


   > @ruanhang1993 Many thanks for finding issues of this PR.
   > I tried to solve two issues you raised.
   > 
   > 1. Regarding the first issue, I relocate protobuf package in pom.xml
   > 2. I'm not sure that open method will be called or not during checkpoint recovery process. @libenchao Could you help answer this question?
   
   @maosuhan  I am honored to receive your reply. I use the fixed version for problem 1 in flink 1.12.1 recently.
   In flink 1.12.1,  the problem 2 is gone. The `open` method will be called in flink 1.12.1 during checkpoint recovery process.
   
   With the fixed code for problem 1 in flink 1.12.1, I got the error like this:
   ```
   Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to org.apache.flink.formats.protobuf.shaded.com.google.protobuf.Descriptors$Descriptor
           at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
           at org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema.<init>(PbRowDataDeserializationSchema.java:67) ~[?:?]
           at org.apache.flink.formats.protobuf.PbDecodingFormat.createRuntimeDecoder(PbDecodingFormat.java:49) ~[?:?]
           at org.apache.flink.formats.protobuf.PbDecodingFormat.createRuntimeDecoder(PbDecodingFormat.java:31) ~[?:?]
           at org.apache.flink.streaming.connectors.kafka.table.KafkaDynamicSource.createDeserialization(KafkaDynamicSource.java:427) ~[?:?]
           ......
   ```
   
   I generate the protobuf message class and package like this, and use it when submitting the flink job.
   ```bash
   protoc -I=$PROTO_FILE_PATH --java_out=$JAVA_OUTPUT/src $PROTO_FILE
   
javac -cp $JAVA_OUTPUT/libs/protobuf-java.jar -d $JAVA_OUTPUT/target $JAVA_OUTPUT/src/$PACKAGE_PATH/*.java


   cd $JAVA_OUTPUT

   jar -cvf $JAR_NAME -C $JAVA_OUTPUT/target/ .
   ```
   
   The problem is that the `getDescriptor` in my class return the `com.google.protobuf.Descriptors$Descriptor` class,  which can not cast to the relocated class.
   ```java
   public static Descriptors.Descriptor getDescriptor(String className) {
    
       try {
        
           Class<?> pbClass = Class.forName(className);
        
           return (Descriptors.Descriptor)
 pbClass.getMethod(PbConstant.PB_METHOD_GET_DESCRIPTOR).invoke(null);
    
       } catch (Exception y) {
        
           throw new IllegalArgumentException(
String.format("get %s descriptors error!", className), y);
    
       }
   
}
   ```
   
   Do I need to relocate the class like you when generating the protobuf message class? 
   Or is there some other way to fix it?
   
   ps: The setting in `META-INF/services/org.apache.flink.table.factories.Factory` needs to be changed.


----------------------------------------------------------------
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



[GitHub] [flink] laughingman7743 commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r652332960



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.AbstractMessage;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * {@link RowToProtoConverter} can convert flink row data to binary protobuf message data by codegen
+ * process.
+ */
+public class RowToProtoConverter {
+    private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+
+    private final Method encodeMethod;
+
+    public RowToProtoConverter(RowType rowType, PbFormatConfig formatConfig)
+            throws PbCodegenException {
+        try {
+            Descriptors.Descriptor descriptor =
+                    PbFormatUtils.getDescriptor(formatConfig.getMessageClassName());
+            Class<?> messageClass = Class.forName(formatConfig.getMessageClassName());
+
+            PbCodegenAppender codegenAppender = new PbCodegenAppender();
+            String uuid = UUID.randomUUID().toString().replaceAll("\\-", "");
+            String generatedClassName = "GeneratedRowToProto_" + uuid;
+            String generatedPackageName = RowToProtoConverter.class.getPackage().getName();
+            codegenAppender.appendLine("package " + generatedPackageName);
+            codegenAppender.appendLine("import " + AbstractMessage.class.getName());
+            codegenAppender.appendLine("import " + Descriptors.class.getName());
+            codegenAppender.appendLine("import " + RowData.class.getName());
+            codegenAppender.appendLine("import " + ArrayData.class.getName());
+            codegenAppender.appendLine("import " + StringData.class.getName());
+            codegenAppender.appendLine("import " + ByteString.class.getName());
+            codegenAppender.appendLine("import " + List.class.getName());
+            codegenAppender.appendLine("import " + ArrayList.class.getName());
+            codegenAppender.appendLine("import " + Map.class.getName());
+            codegenAppender.appendLine("import " + HashMap.class.getName());
+
+            codegenAppender.appendSegment("public class " + generatedClassName + "{");
+            codegenAppender.appendSegment(
+                    "public static AbstractMessage "
+                            + PbConstant.GENERATED_ENCODE_METHOD
+                            + "(RowData rowData){");
+            codegenAppender.appendLine("AbstractMessage message = null");
+            PbCodegenSerializer codegenSer =
+                    PbCodegenSerializeFactory.getPbCodegenTopRowSer(
+                            descriptor, rowType, formatConfig);
+            String genCode = codegenSer.codegen("message", "rowData");
+            codegenAppender.appendSegment(genCode);
+            codegenAppender.appendLine("return message");
+            codegenAppender.appendSegment("}");
+            codegenAppender.appendSegment("}");
+
+            String printCode = codegenAppender.printWithLineNumber();
+            LOG.debug("Protobuf encode codegen: \n" + printCode);
+            Class generatedClass =
+                    PbCodegenUtils.compileClass(
+                            this.getClass().getClassLoader(),

Review comment:
       I haven't experienced any particular errors with this ProtoBuf formatter, but I was curious about it because I have experienced `NoClassDefFoundError` and `ClassNotFoundException` related to Flink's class loader in the past.
   I'm not familiar with Flink's class loader. If the current implementation looks fine, I think it's fine to leave it as is.




-- 
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



[GitHub] [flink] ruanhang1993 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-790247456


   > @ruanhang1993 Many thanks for finding issues of this PR.
   > I tried to solve two issues you raised.
   > 
   > 1. Regarding the first issue, I relocate protobuf package in pom.xml
   > 2. I'm not sure that open method will be called or not during checkpoint recovery process. @libenchao Could you help answer this question?
   
   @maosuhan  I am honored to receive your reply. I use the fixed version for problem 1 in flink 1.12.1 recently.
   In flink 1.12.1,  the problem 2 is gone. The `open` method will be called in flink 1.12.1 during checkpoint recovery process.
   
   With the fixed code for problem 1 in flink 1.12.1, I got the error like this:
   ```
   Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to org.apache.flink.formats.protobuf.shaded.com.google.protobuf.Descriptors$Descriptor
           at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
           at org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema.<init>(PbRowDataDeserializationSchema.java:67) ~[?:?]
           at org.apache.flink.formats.protobuf.PbDecodingFormat.createRuntimeDecoder(PbDecodingFormat.java:49) ~[?:?]
           at org.apache.flink.formats.protobuf.PbDecodingFormat.createRuntimeDecoder(PbDecodingFormat.java:31) ~[?:?]
           at org.apache.flink.streaming.connectors.kafka.table.KafkaDynamicSource.createDeserialization(KafkaDynamicSource.java:427) ~[?:?]
           ......
   ```
   
   I generate the protobuf message class and package like this, and use it when submitting the flink job.
   ```bash
   protoc -I=$PROTO_FILE_PATH --java_out=$JAVA_OUTPUT/src $PROTO_FILE
   
javac -cp $JAVA_OUTPUT/libs/protobuf-java.jar -d $JAVA_OUTPUT/target $JAVA_OUTPUT/src/$PACKAGE_PATH/*.java


   cd $JAVA_OUTPUT

   jar -cvf $JAR_NAME -C $JAVA_OUTPUT/target/ .
   ```
   
   The problem is that the `getDescriptor` in my class return the `com.google.protobuf.Descriptors$Descriptor` class,  which can not cast to the relocated class.
   ```java
   public static Descriptors.Descriptor getDescriptor(String className) {
    
       try {
        
           Class<?> pbClass = Class.forName(className);
        
           return (Descriptors.Descriptor)
 pbClass.getMethod(PbConstant.PB_METHOD_GET_DESCRIPTOR).invoke(null);
    
       } catch (Exception y) {
        
           throw new IllegalArgumentException(
String.format("get %s descriptors error!", className), y);
    
       }
   
}
   ```
   
   Do I need to relocate the class like you when generating the protobuf message class? 
   Or is there some other way to fix it?


----------------------------------------------------------------
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



[GitHub] [flink] ruanhang1993 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-791176342


   There is no problem when packaging flink project. The problem occurs when submitting job by flink cli. 
   I get the exception without the provided scope, like this. The test job aims to write data from kafka to hive.
   ```java
   Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to com.google.protobuf.Descriptors$Descriptor
           at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
          ......
   ```
   The flink directory `lib` contains the `flink-dist` jar(protobuf 3.11.1) and `flink-sql-connector-hive-1.2.2_2.11` jar(protobuf 2.5.0 relocated by me). The `flink-protobuf` jar(protobuf 3.11.1) is in my job jar. And submit job by this command:
   ```bash
   flink run  -m  yarn-cluster  \
   -yd  ...... -yt  protobufMessage.jar  \
   -c  package.Main  myJob.jar  jobParams \
   ```
   ---------------------------------------------------------------------------
   After a few tests, I think the problem is about the class loading in flink, not conflicting with other modules as I thought. 
   
   I need to place the `flink-protobuf` jar under the `lib` dictionary like other formats, e.g. `flink-json`. And every problem is gone. We don't need to change the version in `flink-protobuf` to `protoc.version` or relocate it in `flink-sql-connector-hive`.
   
   It seems that I use the jar in a wrong way. Thanks a lot for your answer.


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-816481162


   @sv3ndk Thanks for your suggestion.
   It seems we can use `com.google.protobuf.Descriptors.FieldDescriptor.hasDefaultValue` to check whether the default value is set of a field.
   this proposal looks good to me. Could you review @sv3ndk 's suggestion and give some feedback. If the proposal is accepted, I will work with @sv3ndk to merge his code to this PR. @libenchao @wuchong 


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ec0895606f7b5cf68a6421be978b4e6fa30e838f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844) 
   
   <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



[GitHub] [flink] sv3ndk commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
sv3ndk commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-809618142


   Thanks a lot for working on a Protobuf format! We also use that serialization mechanism at my company and will need such format  soon, this is very useful.
   
   I have a different understanding regarding the handling of missing values by protobuf 3 and I believe we should be able to provide Fllink with null values in case of missing value in pb. @maosuhan, if you want I'd be happy to collaborate with on on this and add support for this case.
   
   As I understand it:
   
   * protobuf 3 considers every field as optional
   * if a pb field is a complex type, the generated java code contains a `hasXYZ()` method to determine if that field is present
   * if a pb field is a scalar, no such method gets generated
   * when a pb field value is not specified in a pb instance, the `getXYZ()` method returns the default value (i.e `""` instead of `null` in case of string)
   
   The way we approach this where I work is:
   * we only only use protobuf scalar types for non nullable fields
   * we rely on wrappers (like `google.protobuf.StringValue`) for nullable java primitives
   * when reading data, we always check `hasXYZ()` before calling `getXYZ()`
   
   Here's a quick example, generating java classes with pb `3.15.0`:
   
   Given this schema:
   ```
   syntax = "proto3";
   import "google/protobuf/wrappers.proto";
   
   message Parent {
     string id = 1;
     google.protobuf.StringValue name = 2;
     Inner innerValue = 3;
   
     message Inner {
       string f1 = 1;
       string f2 = 2;
     }
   }
   ```
   
   and this `roundTrip()` method:
   
   ```java
     Parent roundTrip(Parent parent) throws InvalidProtocolBufferException {
       return Parent.parser().parseFrom(parent.toByteArray());
     }
   ```
   
   Those assertions show that the missing `name` field can be correctly interpreted both before and after serialization:
   
   ```java
    var withoutName = Parent.newBuilder()
           .setId("theId")
           .setInnerValue(
               Parent.Inner.newBuilder()
                   .setF1("theF1")
                   .setF2("theF1")
                   .build()
           )
           .build();
   
       assert ! withoutName.hasName();
       assert ! roundTrip(withoutName).hasName();
       assert withoutName.hasInnerValue();
       assert roundTrip(withoutName).hasInnerValue();
       assert withoutName.getInnerValue().getF1().equals("theF1");
       assert roundTrip(withoutName).getInnerValue().getF1().equals("theF1");
       //assert ! hasNoName.hasId();           // does not compile: hasXYZ() does not exist if XYZ is a scalar
   ```
   
   Similarly, this instance with a missing nested field can be interpreted correctly as well by a reader:
   
   ```java
    var withoutInner = Parent.newBuilder()
           .setId("theId")
           .setName(StringValue.newBuilder().setValue("theName").build())
           .build();
   
       assert ! withoutInner.hasInnerValue();
       assert ! roundTrip(withoutInner).hasInnerValue();
       assert withoutInner.hasName();
       assert roundTrip(withoutInner).hasName();
       assert withoutInner.getName().getValue().equals("theName");
       assert roundTrip(withoutInner).getName().getValue().equals("theName");
   ```
   


-- 
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



[GitHub] [flink] sv3ndk edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
sv3ndk edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-809618142


   Thanks a lot for working on a Protobuf format! We also use that serialization mechanism at my company and will need such format  soon, this is very useful.
   
   I have a different understanding regarding the handling of missing values by protobuf 3 and I believe we should be able to provide Fllink with nulls in case of missing value in pb. @maosuhan, if you want I'd be happy to collaborate with you on on this and add support for this case.
   
   As I understand it:
   
   * protobuf 3 considers every field as optional
   * if a pb field is a complex type, the generated java code contains a `hasXYZ()` method to determine if that field is present
   * if a pb field is a scalar, no such method gets generated
   * when a pb field value is not specified in a pb instance, the `getXYZ()` method returns the default value (i.e `""` instead of `null` in case of string)
   
   The way we approach this where I work is:
   * we only only use protobuf scalar types for non nullable fields
   * we rely on wrappers (like `google.protobuf.StringValue`) for nullable java primitives
   * when reading data, we always check `hasXYZ()` before calling `getXYZ()`
   
   Here's a quick example, generating java classes with pb `3.15.0`:
   
   Given this schema:
   ```
   syntax = "proto3";
   import "google/protobuf/wrappers.proto";
   
   message Parent {
     string id = 1;
     google.protobuf.StringValue name = 2;
     Inner innerValue = 3;
   
     message Inner {
       string f1 = 1;
       string f2 = 2;
     }
   }
   ```
   
   and this `roundTrip()` method:
   
   ```java
     Parent roundTrip(Parent parent) throws InvalidProtocolBufferException {
       return Parent.parser().parseFrom(parent.toByteArray());
     }
   ```
   
   Those assertions show that the missing `name` field can be correctly interpreted both before and after serialization:
   
   ```java
    var withoutName = Parent.newBuilder()
           .setId("theId")
           .setInnerValue(
               Parent.Inner.newBuilder()
                   .setF1("theF1")
                   .setF2("theF1")
                   .build()
           )
           .build();
   
       assert ! withoutName.hasName();
       assert ! roundTrip(withoutName).hasName();
       assert withoutName.hasInnerValue();
       assert roundTrip(withoutName).hasInnerValue();
       assert withoutName.getInnerValue().getF1().equals("theF1");
       assert roundTrip(withoutName).getInnerValue().getF1().equals("theF1");
       //assert ! hasNoName.hasId();           // does not compile: hasXYZ() does not exist if XYZ is a scalar
   ```
   
   Similarly, this instance with a missing nested field can be interpreted correctly as well by a reader:
   
   ```java
    var withoutInner = Parent.newBuilder()
           .setId("theId")
           .setName(StringValue.newBuilder().setValue("theName").build())
           .build();
   
       assert ! withoutInner.hasInnerValue();
       assert ! roundTrip(withoutInner).hasInnerValue();
       assert withoutInner.hasName();
       assert roundTrip(withoutInner).hasName();
       assert withoutInner.getName().getValue().equals("theName");
       assert roundTrip(withoutInner).getName().getValue().equals("theName");
   ```
   


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860) 
   * a9a50e78d0ba6ff84d02fbadee1484970fac2c79 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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 058a86bf55557a420b625c0a96f2a4cf0a323fe3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * d7b612acb72c25b4a30899155b764b6ad2535613 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612) 
   
   <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



[GitHub] [flink] laughingman7743 commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r611049891



##########
File path: flink-formats/flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
##########
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

Review comment:
       ```suggestion
   # 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.
   ```
   When I ran the test, I got the following error:
   ```
   2021-04-10 23:05:10,066 ERROR  ||  Could not load service provider for factories.   [org.apache.flink.table.factories.FactoryUtil]
   java.util.ServiceConfigurationError: org.apache.flink.table.factories.Factory: file:/path/to/target/scala-2.12/classes/META-INF/services/org.apache.flink.table.factories.Factory:1: Illegal provider-class name: /*
   	at java.base/java.util.ServiceLoader.fail(ServiceLoader.java:588)
   	at java.base/java.util.ServiceLoader.fail(ServiceLoader.java:594)
   	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.parseLine(ServiceLoader.java:1142)
   	at java.base/java.util.ServiceLoader$LazyClassPathLookupIterator.parse(ServiceLoader.java:1169)
   ```
   Perhaps the comments in this file start with `#`.
   https://github.com/apache/flink/blob/release-1.12.2/flink-formats/flink-json/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory#L1-L14




-- 
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



[GitHub] [flink] flinkbot commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744243277


   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 4b702caaac7b53343401aee15f02454fd7a6f791 (Mon Dec 14 07:43:33 UTC 2020)
   
   **Warnings:**
    * **2 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-18202).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r600964583



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()
+			.equals(descriptor.getFile().getPackage())) {
+			throw new IllegalArgumentException(
+				"java_package and package must be the same in proto definition");
+		}
+		if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       Now the PR can support different combinations of java_package, package, java_outer_class_name, java_multiple_files now.




-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * bd27dabd13c11fd65b98423595e9d93234538444 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330) 
   * e3f953d91778ef96ad5fe222dd9da229dc2c5619 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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c6f6ae6aec6f3bed305b56b6634422af8fd8da7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900) 
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "27bec8aa19f0b1d530a3be3d736fcd2c9a032446",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "27bec8aa19f0b1d530a3be3d736fcd2c9a032446",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   * 27bec8aa19f0b1d530a3be3d736fcd2c9a032446 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   * 888452faa35e98b1151416484edfcf9e097f680f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] complone removed a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
complone removed a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-943001370


   @wuchong This task does not seem to be progressing at the moment. May I assign this task to me? We currently have a need to support protobuf-format


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-974770663


   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-982317036


   @MyLanPangzi Thanks for your feedback. I think it is doable, what about adding a new option like 'protobuf.enum-as-int'?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765






-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zhuzhengjun01 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
zhuzhengjun01 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1016191275


   > 
   like this , 
   `
   syntax = proto3;
   package org.apache.flink.formats.protobuf.proto;
   option java_multiple_files = false;
   
   message KafkaSinkTestTT {
       oneof name_oneof {
           string name = 1;
   
       }
       oneof num_oneof {
           int32 num = 2;
       }
       oneof tt_oneof {
           int64 xtime = 3;
   
       }
       ro row1 = 4;
       message ro{
           string a = 1;
           int32 b = 2;
       }
   }
   `
   actually, I want to distinguish between default and missing values in proto3
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1016185722


   > @maosuhan hi, flink-protobuf works very well in my job, but i found some corner case that it was not supported `oneof` in proto3. so this will be dealt in future or not ?
   
   Thanks for your feedback. Could you provide your proto definition?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] libenchao commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
libenchao commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-805772483


   @wuchong What's your opinion for these issues?


-- 
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



[GitHub] [flink] sv3ndk commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
sv3ndk commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-826620751


   Hi all,
   
   Can I please collect your feed-back on my proposal for updating the current PR?
   
   What i suggest is simply to remove the configuration flag `readDefaultValues` and just use the nominal behavior of protobuf regarding to default values, s.t. this format is aligned with protobuf spec. 
   In more details, the behavior regarding nulls and missing values would be the one described in this comment: https://github.com/apache/flink/pull/14376#issuecomment-814650640
   
   I have not pushed the code to do that anywhere yet, although I have on my local setup and so far it seems to work fine.
   
   What do you think?
   


-- 
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



[GitHub] [flink] ruanhang1993 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-786494390


   Hi, @maosuhan thanks a lot for your work. I have used your code in flink 1.11.2. There seems to be some problems for me:
   
   1. The protobuf-java jar in flink-protobuf module conflicts with flink-dist module and flink-sql-connector-hive module.
   2. NullPointerException for field `protoToRowConverter` when recovering from some checkpoint
   
   For problem 1, I set the scope of the protobuf-java in flink-protobuf module to `provided`, and use the version setting by `protoc.version` in flink-parent. Besides, I relocated the protobuf-java in fink-sql-connector-hive.
   ```xml
   <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
       <version>${protoc.version}</version>
       <scope>provided</scope>
   </dependency>
   ```
   
   For problem 2, it seems that the `open` method in `PbRowDeserializationSchema` is not called when recovering from some checkpoint(submit job by cli `flink run -s <dir>`), which causes the field `protoToRowConverter` to be null. I fixed it by the following code in `PbRowDeserializationSchema.java`.
   ```java
   @Override
   public RowData deserialize(byte[] message) throws IOException {
       try {
           if(protoToRowConverter == null) {
               LOG.info(String.format("[protobuf new]: %s %s %s", messageClassName, rowType.toString(), "" + readDefaultValues));
               protoToRowConverter = new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
           }
           return protoToRowConverter.convertProtoBinaryToRow(message);
       } catch (Throwable t) {
           if (ignoreParseErrors) {
               return null;
           }
           LOG.error("Failed to deserialize PB object.", t);
           throw new IOException("Failed to deserialize PB object.", t);
       }
   }
   ```
   
   Will the same problems occur in the new flink version ?
   And do you have any better ideas to fix it ?
   
   Thanks for reading.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a9a50e78d0ba6ff84d02fbadee1484970fac2c79 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709) 
   * bbfc0d96eb419932d49a54bf95f008c3155fbc81 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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r652323821



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+
+import com.google.protobuf.Descriptors;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    /**
+     * protobuf code has a bug that, f_abc_7d will be convert to fAbc7d in {@code
+     * com.google.protobuf.Descriptors.FileDescriptor.getJsonName()}, but actually we need fAbc7D.
+     */
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                if ('a' <= ch && ch <= 'z') {
+                    ch = (char) (ch - 'a' + 'A');
+                    isNextUpperCase = false;
+                }
+                result.append(ch);
+            } else {
+                result.append(ch);
+            }
+        }
+        return result.toString();
+    }
+
+    public static String getFullJavaName(Descriptors.Descriptor descriptor) {
+        String javaPackageName = descriptor.getFile().getOptions().getJavaPackage();

Review comment:
       That's good finding. I have fixed it. There's a problem that javaPackageName can be specified as "", so we should use descriptor.getFile().getOptions().hasJavaPackage instead




-- 
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



[GitHub] [flink] complone edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
complone edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-966850088


   @wuchong @libenchao @maosuhan 
   Hello, everyone. Our company is working on the unfinished protobuf-format. I noticed that this featrue has not been in progress for three months, and for some common scenarios (such as canal's message.proto), whether to assign this task to me ?
   
   CanalPbMessageTest:
   https://github.com/complone/flink-format-flink/blob/main/src/test/java/com/alibaba/otter/canal/protocol/CanalPbMessageTest.java
   message.proto
   
   ```
   syntax = "proto3";
   
   package com.alibaba.otter.canal.protocol;
   
   option java_package = "com.alibaba.otter.canal.protocol.entity";
   option java_multiple_files = true;
   //option java_outer_classname = "MessageL";
   option optimize_for = SPEED;
   option cc_generic_services = true;
   
   //还原出的canal消息格式
   message Message{
     int64 id = 1;
     repeated CanalEntry.Entry entries = 2;
     bool raw = 3;
     repeated bytes rawEntries = 4;
   }
   
   message CanalEntry{
     message Entry{
       /**协议头部信息 **/
       optional Header header = 1;
   
       /** 需要序列化的类型为ROWDATA类型**/
       oneof entryType_present{
         EntryType entryType = 2;
       }
       /**传输的二进制组 **/
       bytes storeValue = 3;
     }
   }
   
   /** 事件类型**/
   enum EventType{
     EVENTTYPECOMPATIBLEPROTO2 = 0;
     INSERT 		= 		1;
     UPDATE 		= 		2;
     DELETE 		= 		3;
     CREATE		= 		4;
     ALTER		= 		5;
     ERASE		= 		6;
     QUERY		=		7;
     TRUNCATE	=		8;
     RENAME 		= 		9;
     /**CREATE INDEX**/
     CINDEX		= 		10;
     DINDEX 		= 		11;
     GTID        =       12;
     /** XA **/
     XACOMMIT    =       13;
     XAROLLBACK  =		14;
     /** MASTER HEARTBEAT **/
     MHEARTBEAT  =       15;
   }
   
   /**打散后的事件类型,主要用于标识事务的开始,变更数据,结束**/
   enum EntryType{
     ENTRYTYPECOMPATIBLEPROTO2 = 0;
     TRANSACTIONBEGIN 		=		1;
     ROWDATA					=		2;
     TRANSACTIONEND			=		3;
     /** 心跳类型,内部使用,外部暂不可见,可忽略 **/
     HEARTBEAT				=		4;
     GTIDLOG                 =       5;
   }
   
   
   
   /**每个字段的数据结构**/
   message Column {
     /**字段下标 UPDATE DELETE **/
     int32 index = 1;
   
     /**字段java类型 **/
     int32 sqlType = 2;
     /**字段名称 **/
     string name = 3;
   }
   
   
   /**数据包格式**/
   message RowData{
   
     /**字段信息,增量数据(修改后,新增后)**/
     repeated Column afterColumns = 2;
   
     /**字段信息,增量数据(修改前,删除前)**/
     /**changelog insert,delete,**/
     repeated Column beforeColumns = 1;
   
     /**预留拓展 **/
     repeated Pair pros = 3;
   }
   
   /**message row每行数据变更的数据结构 **/
   message RowChange{
   
     /**tableId,由数据库产生 **/
     int64 tableId = 1;
     oneof eventType_present {
       EventType eventType = 2;
       
     }
   
   }
   
   /**预留嵌套消息拓展 **/
   message Pair{
     string key = 1;
     string value = 2;
   }
   message Header{
     oneof version_present {
       int32 version = 1;
     }
   
     /**binlog/redolog文件名 **/
     string logFileName = 2;
   
     /**binlog/redolog文件的偏移位置 **/
     int64 logFileOffset = 3;
     /**服务端serverId **/
     int64 serverId = 4;
     /**变更数据的执行编码 **/
     int64 serverenCode = 5;
     /**变更数据的执行时间**/
     int64 executeTime = 6;
     /**变更数据的来源 **/
     oneof sourceType_present{
       Type sourceType = 7;
     }
   
     /**变更数据的schemaname **/
     string schemaName = 8;
   
     /**变更数据的tablename **/
     string tableName = 9;
     /**传输数据的事件长度**/
     int64 eventLength = 10;
     /**数据变更类型 **/
     oneof eventType_present{
       EventType eventType = 11;
     }
   
     /**预留扩展**/
     repeated Pair					props				= 12;
   
     /**当前事务的gitd**/
     string                 gtid                = 13;
   
     /**上游需要监听的数据源类型 **/
    enum Type{
      TYPECOMPATIBLEPROTO2 = 0;
      ORACLE = 1;
      MYSQL = 2;
      PGSQL = 3;
    }
   }
   ```


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-971329553


   @maosuhan I can find a committer for a review. I do see two things currently standing out:
   
   1. It would be good to make sure that the commits are meeting the commit requirements, as per https://flink.apache.org/contributing/code-style-and-quality-pull-requests.html 
   2. The CI is not yet green due to a couple of issues. Have you seen them and can you resolve them?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-978722264


   @MartijnVisser I think so, I created `class PbRowDataDeserializationSchema implements DeserializationSchema<RowData>` and `class PbRowDataSerializationSchema implements SerializationSchema<RowData>` so that it can be used in DataStream and Table API code. I'm not sure this is what you want?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-972979037


   @MartijnVisser Where can I find the information of CI?  It will be helpful if you can send the link.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * 0000 Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-972991351


   @maosuhan You can find the links here https://github.com/apache/flink/pull/14376#issuecomment-744252765


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 12bc9ca2640c7773b2ca7fa50e204605183ff309 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MyLanPangzi commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MyLanPangzi commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r764575993



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.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.protobuf;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.google.protobuf.Descriptors;
+import org.apache.commons.lang3.StringUtils;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    /**
+     * protobuf code has a bug that, f_abc_7d will be convert to fAbc7d in {@code
+     * com.google.protobuf.Descriptors.FileDescriptor.getJsonName()}, but actually we need fAbc7D.
+     */
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                if ('a' <= ch && ch <= 'z') {
+                    ch = (char) (ch - 'a' + 'A');
+                    isNextUpperCase = false;
+                }
+                result.append(ch);
+            } else {
+                result.append(ch);
+            }
+        }
+        return result.toString();
+    }
+
+    private static String getJavaPackageFromProtoFile(Descriptors.Descriptor descriptor) {
+        boolean hasJavaPackage = descriptor.getFile().getOptions().hasJavaPackage();
+        if (hasJavaPackage) {
+            String javaPackage = descriptor.getFile().getOptions().getJavaPackage();
+            if (StringUtils.isBlank(javaPackage)) {
+                throw new FlinkRuntimeException("java_package cannot be blank string");
+            }
+            return javaPackage;
+        } else {
+            String packageName = descriptor.getFile().getPackage();
+            if (StringUtils.isBlank(packageName)) {
+                throw new FlinkRuntimeException("package and java_package cannot both be empty");
+            }
+            return packageName;
+        }
+    }
+
+    public static String getFullJavaName(Descriptors.Descriptor descriptor) {
+        String javaPackageName = getJavaPackageFromProtoFile(descriptor);
+        if (descriptor.getFile().getOptions().getJavaMultipleFiles()) {
+            // multiple_files=true
+            if (null != descriptor.getContainingType()) {
+                // nested type
+                String parentJavaFullName = getFullJavaName(descriptor.getContainingType());
+                return parentJavaFullName + "." + descriptor.getName();
+            } else {
+                // top level message
+                return javaPackageName + "." + descriptor.getName();
+            }
+        } else {
+            // multiple_files=false
+            if (null != descriptor.getContainingType()) {
+                // nested type
+                String parentJavaFullName = getFullJavaName(descriptor.getContainingType());
+                return parentJavaFullName + "." + descriptor.getName();
+            } else {
+                // top level message
+                if (!descriptor.getFile().getOptions().hasJavaOuterClassname()) {
+                    // user do not define outer class name in proto file
+                    return javaPackageName

Review comment:
       @maosuhan 
   1.I find some bugs. When i don't declare **java_outer_classname=XXX** or **java_multiple_files=true** option in file .proto(proto2)  this will be generated x.y.z.xxxOuterClass.MyPBClass. It is incorrect.
   
   generated code: public static RowData decode(data.LogOuterClass.Log message){
   my pb code data.Data.Log
   
   2.How to identify inner class?




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27985",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27985) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] sv3ndk commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
sv3ndk commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-828208840


   Ok, thanks for letting me know.
   Looking forward to protobuf support in Flink!


-- 
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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-787463110


   @ruanhang1993 Many thanks for finding issues of this PR.
   I tried to solve two issues you raised.
   1. Regarding the first issue, I relocate protobuf package in pom.xml
   2. I'm not sure that open method will be called or not during checkpoint recovery process. @libenchao Could you help answer this question?


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r652326974



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.AbstractMessage;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * {@link RowToProtoConverter} can convert flink row data to binary protobuf message data by codegen
+ * process.
+ */
+public class RowToProtoConverter {
+    private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+
+    private final Method encodeMethod;
+
+    public RowToProtoConverter(RowType rowType, PbFormatConfig formatConfig)
+            throws PbCodegenException {
+        try {
+            Descriptors.Descriptor descriptor =
+                    PbFormatUtils.getDescriptor(formatConfig.getMessageClassName());
+            Class<?> messageClass = Class.forName(formatConfig.getMessageClassName());
+
+            PbCodegenAppender codegenAppender = new PbCodegenAppender();
+            String uuid = UUID.randomUUID().toString().replaceAll("\\-", "");
+            String generatedClassName = "GeneratedRowToProto_" + uuid;
+            String generatedPackageName = RowToProtoConverter.class.getPackage().getName();
+            codegenAppender.appendLine("package " + generatedPackageName);
+            codegenAppender.appendLine("import " + AbstractMessage.class.getName());
+            codegenAppender.appendLine("import " + Descriptors.class.getName());
+            codegenAppender.appendLine("import " + RowData.class.getName());
+            codegenAppender.appendLine("import " + ArrayData.class.getName());
+            codegenAppender.appendLine("import " + StringData.class.getName());
+            codegenAppender.appendLine("import " + ByteString.class.getName());
+            codegenAppender.appendLine("import " + List.class.getName());
+            codegenAppender.appendLine("import " + ArrayList.class.getName());
+            codegenAppender.appendLine("import " + Map.class.getName());
+            codegenAppender.appendLine("import " + HashMap.class.getName());
+
+            codegenAppender.appendSegment("public class " + generatedClassName + "{");
+            codegenAppender.appendSegment(
+                    "public static AbstractMessage "
+                            + PbConstant.GENERATED_ENCODE_METHOD
+                            + "(RowData rowData){");
+            codegenAppender.appendLine("AbstractMessage message = null");
+            PbCodegenSerializer codegenSer =
+                    PbCodegenSerializeFactory.getPbCodegenTopRowSer(
+                            descriptor, rowType, formatConfig);
+            String genCode = codegenSer.codegen("message", "rowData");
+            codegenAppender.appendSegment(genCode);
+            codegenAppender.appendLine("return message");
+            codegenAppender.appendSegment("}");
+            codegenAppender.appendSegment("}");
+
+            String printCode = codegenAppender.printWithLineNumber();
+            LOG.debug("Protobuf encode codegen: \n" + printCode);
+            Class generatedClass =
+                    PbCodegenUtils.compileClass(
+                            this.getClass().getClassLoader(),

Review comment:
       Thanks for your comment. What if I use this.getClass().getClassLoader() to load the generated class? What is the difference of InitializationContext.UserCodeClassLoader ?




-- 
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



[GitHub] [flink] laughingman7743 commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r649007396



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+
+import com.google.protobuf.Descriptors;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    /**
+     * protobuf code has a bug that, f_abc_7d will be convert to fAbc7d in {@code
+     * com.google.protobuf.Descriptors.FileDescriptor.getJsonName()}, but actually we need fAbc7D.
+     */
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                if ('a' <= ch && ch <= 'z') {
+                    ch = (char) (ch - 'a' + 'A');
+                    isNextUpperCase = false;
+                }
+                result.append(ch);
+            } else {
+                result.append(ch);
+            }
+        }
+        return result.toString();
+    }
+
+    public static String getFullJavaName(Descriptors.Descriptor descriptor) {
+        String javaPackageName = descriptor.getFile().getOptions().getJavaPackage();

Review comment:
       It seems that `javaPackageName` is empty in some cases.
   In this case, it is better to use `descriptor.getFile().getPackage()` to get the java package name.
   ```suggestion
           String javaPackageName = descriptor.getFile().getOptions().getJavaPackage();
           if (javaPackageName.isEmpty()) {
               javaPackageName = descriptor.getFile().getPackage();
           }
   ```




-- 
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



[GitHub] [flink] ruanhang1993 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-791176342


   There is no problem when packaging flink project. The problem occurs when submitting job by flink cli. 
   I get the exception without the provided scope, like this. The test job aims to write data from kafka to hive.
   ```java
   Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to com.google.protobuf.Descriptors$Descriptor
           at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
          ......
   ```
   The flink directory `lib` contains the `flink-dist` jar(protobuf 3.11.1) and `flink-sql-connector-hive-1.2.2_2.11` jar(protobuf 2.5.0 relocated by me). The `flink-protobuf` jar(protobuf 3.11.1) is in my job jar. And submit job by this command:
   ```bash
   flink run  -m  yarn-cluster  \
   -yd  ...... -yt  protobufMessage.jar  \
   -c  package.Main  myJob.jar  jobParams \
   ```
   ---------------------------------------------------------------------------
   After a few tests, I think the problem is about the class loading in flink, not conflicting with other modules as I thought. 
   
   I need to place the `flink-protobuf` jar under the `lib` directory like other formats, e.g. `flink-json`. And every problem is gone. We don't need to change the version in `flink-protobuf` to `protoc.version` or relocate it in `flink-sql-connector-hive`.
   
   It seems that I use the jar in a wrong way. Thanks a lot for your answer.


----------------------------------------------------------------
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



[GitHub] [flink] syucream commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
syucream commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r547039824



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenSimpleDeserializer.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.protobuf.deserialize;
+
+import com.google.protobuf.Descriptors;
+
+public class PbCodegenSimpleDeserializer implements PbCodegenDeserializer {
+	private Descriptors.FieldDescriptor fd;
+
+	public PbCodegenSimpleDeserializer(
+		Descriptors.FieldDescriptor fd) {
+		this.fd = fd;
+	}
+
+	@Override
+	public String codegen(String returnVarName, String messageGetStr) {
+		StringBuilder sb = new StringBuilder();
+		switch (fd.getJavaType()) {
+			case INT:
+			case LONG:
+			case FLOAT:
+			case DOUBLE:
+			case BOOLEAN:
+				sb.append(returnVarName + " = " + messageGetStr + ";");
+				break;
+			case BYTE_STRING:
+				sb.append(returnVarName + " = " + messageGetStr + ".toByteArray();");
+				break;
+			case STRING:
+			case ENUM:
+				sb.append(
+					returnVarName + " = StringData.fromString(" + messageGetStr + ".toString());");

Review comment:
       It raises `java.lang.IncompatibleClassChangeError` when I built with Java11:
   ```
   java.io.IOException: Failed to deserialize PB object.
           at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:96)
           at org.apache.flink.formats.protobuf.MapProtoToRowTest.testMessage(MapProtoToRowTest.java:52)
           at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.base/java.lang.reflect.Method.invoke(Method.java:566)
           at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
           at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
           at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
           at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
           at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
           at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
           at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
           at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
           at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
           at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
           at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
           at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
           at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
           at org.junit.runners.Suite.runChild(Suite.java:128)
           at org.junit.runners.Suite.runChild(Suite.java:27)
           at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
           at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
           at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
           at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
           at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
           at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
           at org.apache.maven.surefire.junitcore.JUnitCore.run(JUnitCore.java:55)
           at org.apache.maven.surefire.junitcore.JUnitCoreWrapper.createRequestAndRun(JUnitCoreWrapper.java:137)
           at org.apache.maven.surefire.junitcore.JUnitCoreWrapper.executeLazy(JUnitCoreWrapper.java:119)
           at org.apache.maven.surefire.junitcore.JUnitCoreWrapper.execute(JUnitCoreWrapper.java:87)
           at org.apache.maven.surefire.junitcore.JUnitCoreWrapper.execute(JUnitCoreWrapper.java:75)
           at org.apache.maven.surefire.junitcore.JUnitCoreProvider.invoke(JUnitCoreProvider.java:158)
           at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
           at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
           at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
           at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
   Caused by: java.lang.reflect.InvocationTargetException
           at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.base/java.lang.reflect.Method.invoke(Method.java:566)
           at org.codehaus.janino.ScriptEvaluator.evaluate(ScriptEvaluator.java:798)
           at org.codehaus.janino.ScriptEvaluator.evaluate(ScriptEvaluator.java:790)
           at org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter.convertProtoBinaryToRow(ProtoToRowConverter.java:97)
           at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:90)
           ... 36 more
   Caused by: java.lang.IncompatibleClassChangeError: Method 'org.apache.flink.table.data.StringData org.apache.flink.table.data.StringData.fromString(java.lang.String)' must be InterfaceMethodref constant
           at SC.eval0(Unknown Source)
           ... 44 more
   ```
   
   In my guess, the cause is Janino's limitation https://github.com/janino-compiler/janino/issues/69#issuecomment-707620587, and I created a patch for a workaround. What do you think about this? https://github.com/maosuhan/flink/pull/1




----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-798874043


   @libenchao I have fixed check-style errors. They're all related to java doc issue. 
   You can forcibly pull my branch and review again because I have rebased the code on master.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * bd27dabd13c11fd65b98423595e9d93234538444 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 076885377a4766f739fefa258da0b75c98e7a64a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137) 
   
   <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



[GitHub] [flink] ruanhang1993 removed a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 removed a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-786451333


   HI, @maosuhan thanks a lot for your work. I have used the code based on flink 1.11.2. There are some problem for me when using the protobuf format:
   
   1. protobuf-java jar in flink-protobuf conflicts with which in flink-dist and flink-sql-connector-hive
   2. There are some problems using protobuf format when recovering from some checkpoint
   ```
   Caused by: java.io.IOException: Failed to deserialize PB object.
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:97) ~[xxx.jar:?]
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:39) ~[xxx.jar:?]
   	at org.apache.flink.api.common.serialization.DeserializationSchema.deserialize(DeserializationSchema.java:81) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper.deserialize(KafkaDeserializationSchemaWrapper.java:56) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.partitionConsumerRecordsHandler(KafkaFetcher.java:181) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.runFetchLoop(KafkaFetcher.java:141) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755) ~[xxx.jar:?]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:213) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   Caused by: java.lang.NullPointerException
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:91) ~[xxx.jar:?]
   	at org.apache.flink.formats.protobuf.deserialize.PbRowDeserializationSchema.deserialize(PbRowDeserializationSchema.java:39) ~[xxx.jar:?]
   	at org.apache.flink.api.common.serialization.DeserializationSchema.deserialize(DeserializationSchema.java:81) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.connectors.kafka.internals.KafkaDeserializationSchemaWrapper.deserialize(KafkaDeserializationSchemaWrapper.java:56) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.partitionConsumerRecordsHandler(KafkaFetcher.java:181) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.internal.KafkaFetcher.runFetchLoop(KafkaFetcher.java:141) ~[xxx.jar:?]
   	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:755) ~[xxx.jar:?]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:100) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:63) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   	at org.apache.flink.streaming.runtime.tasks.SourceStreamTask$LegacySourceFunctionThread.run(SourceStreamTask.java:213) ~[flink-dist_2.11-1.11.2.jar:1.11.2]
   ```
   
   For problem 1,I resolve it by using the protobuf-java jar in flink-dist,and set the scope of protobuf-java in flink-protobuf to provided. Besides, I shaded the protobuf-java in flink-sql-connector-hive.
   ```
   // flink-protobuf : pom.xml
   <dependency>
   	<groupId>com.google.protobuf</groupId>
   	<artifactId>protobuf-java</artifactId>
   	<version>${protoc.version}</version>
   	<scope>provided</scope>
   </dependency>
   
   // flink-sql-connector-hive-1.2.2 :pom.xml :shade-plugin
   <relocations>
   	<relocation>
   		<pattern>com.google.protobuf</pattern>
   		<shadedPattern>org.apache.flink.connector.sql.hive.com.google.protobuf</shadedPattern>
   	</relocation>
   </relocations>
   ```
   For problem 2 (when submit job by `flink run -s <checkpoint>`), I simply used the following code for `deserialize` method in `PbRowDeserializationSchema` class to fix this issue.
   ```java
   @Override
   public RowData deserialize(byte[] message) throws IOException {
       try {
           if(protoToRowConverter == null) {
               protoToRowConverter = new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
   	}
           return protoToRowConverter.convertProtoBinaryToRow(message);
       } catch (Throwable t) {
               if (ignoreParseErrors) {
                   return null;
               }
               LOG.error("Failed to deserialize PB object.", t);
               throw new IOException("Failed to deserialize PB object.", t);
       }
   }
   ```
   
   I don't know if the question is that the way i use or the different flink version 1.11.2, hope the problems for me are helpful for this PR.
   Do you have any better advice for me to fix it ? 
   
   Thanks for Reading.
   


----------------------------------------------------------------
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



[GitHub] [flink] ruanhang1993 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-791176342


   There is no problem when packaging flink project. The problem occurs when submitting job by flink cli. 
   I get the exception without the provided scope, like this. The test job aims to write data from kafka to hive.
   ```java
   Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to com.google.protobuf.Descriptors$Descriptor
           at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
           at org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema.<init>(PbRowDataDeserializationSchema.java:67) ~[?:?]
           at org.apache.flink.formats.protobuf.PbDecodingFormat.createRuntimeDecoder(PbDecodingFormat.java:49) ~[?:?]
           at org.apache.flink.formats.protobuf.PbDecodingFormat.createRuntimeDecoder(PbDecodingFormat.java:31) ~[?:?]
   ```
   The flink dictionary `lib` contains the `flink-dist` jar(protobuf 3.11.1) and `flink-sql-connector-hive-1.2.2_2.11` jar(protobuf 2.5.0 relocated by me). The `flink-protobuf` jar(protobuf 3.11.1) is in my job jar. And submit job by this command:
   ```bash
   flink run  -m  yarn-cluster  \
   -yd  ...... -yt  protobufMessage.jar  \
   -c  package.Main  myJob.jar  jobParams \
   ```
   ---------------------------------------------------------------------------
   After a few tests, I think the problem is about the class loading in flink, not conflicting with other modules as I thought. 
   
   I need to place the `flink-protobuf` jar under the `lib` dictionary like other formats, e.g. `flink-json`. And every problem is gone. We don't need to change the version in `flink-protobuf` to `protoc.version` or relocate it in `flink-sql-connector-hive`.
   
   It seems that I use the jar in a wrong way. Thanks a lot for your answer.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 7d049bb800be2ae6c53e63fd2917a514dab086dd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006) 
   * 24361517f485cb00e0b85f6e201419da072334b5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] complone commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
complone commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-966850088


   @wuchong @libenchao @maosuhan 
   Hello, everyone. Our company is working on the unfinished protobuf-format. I noticed that this featrue has not been in progress for three months, and for some common scenarios (such as canal's message.proto), whether to assign this task to me ?
   
   message.proto
   
   ```
   syntax = "proto3";
   
   package com.alibaba.otter.canal.protocol;
   
   option java_package = "com.alibaba.otter.canal.protocol.entity";
   option java_multiple_files = true;
   //option java_outer_classname = "MessageL";
   option optimize_for = SPEED;
   option cc_generic_services = true;
   
   //还原出的canal消息格式
   message Message{
     int64 id = 1;
     repeated CanalEntry.Entry entries = 2;
     bool raw = 3;
     repeated bytes rawEntries = 4;
   }
   
   message CanalEntry{
     message Entry{
       /**协议头部信息 **/
       optional Header header = 1;
   
       /** 需要序列化的类型为ROWDATA类型**/
       oneof entryType_present{
         EntryType entryType = 2;
       }
       /**传输的二进制组 **/
       bytes storeValue = 3;
     }
   }
   
   /** 事件类型**/
   enum EventType{
     EVENTTYPECOMPATIBLEPROTO2 = 0;
     INSERT 		= 		1;
     UPDATE 		= 		2;
     DELETE 		= 		3;
     CREATE		= 		4;
     ALTER		= 		5;
     ERASE		= 		6;
     QUERY		=		7;
     TRUNCATE	=		8;
     RENAME 		= 		9;
     /**CREATE INDEX**/
     CINDEX		= 		10;
     DINDEX 		= 		11;
     GTID        =       12;
     /** XA **/
     XACOMMIT    =       13;
     XAROLLBACK  =		14;
     /** MASTER HEARTBEAT **/
     MHEARTBEAT  =       15;
   }
   
   /**打散后的事件类型,主要用于标识事务的开始,变更数据,结束**/
   enum EntryType{
     ENTRYTYPECOMPATIBLEPROTO2 = 0;
     TRANSACTIONBEGIN 		=		1;
     ROWDATA					=		2;
     TRANSACTIONEND			=		3;
     /** 心跳类型,内部使用,外部暂不可见,可忽略 **/
     HEARTBEAT				=		4;
     GTIDLOG                 =       5;
   }
   
   
   
   /**每个字段的数据结构**/
   message Column {
     /**字段下标 UPDATE DELETE **/
     int32 index = 1;
   
     /**字段java类型 **/
     int32 sqlType = 2;
     /**字段名称 **/
     string name = 3;
   }
   
   
   /**数据包格式**/
   message RowData{
   
     /**字段信息,增量数据(修改后,新增后)**/
     repeated Column afterColumns = 2;
   
     /**字段信息,增量数据(修改前,删除前)**/
     /**changelog insert,delete,**/
     repeated Column beforeColumns = 1;
   
     /**预留拓展 **/
     repeated Pair pros = 3;
   }
   
   /**message row每行数据变更的数据结构 **/
   message RowChange{
   
     /**tableId,由数据库产生 **/
     int64 tableId = 1;
     oneof eventType_present {
       EventType eventType = 2;
       
     }
   
   }
   
   /**预留嵌套消息拓展 **/
   message Pair{
     string key = 1;
     string value = 2;
   }
   message Header{
     oneof version_present {
       int32 version = 1;
     }
   
     /**binlog/redolog文件名 **/
     string logFileName = 2;
   
     /**binlog/redolog文件的偏移位置 **/
     int64 logFileOffset = 3;
     /**服务端serverId **/
     int64 serverId = 4;
     /**变更数据的执行编码 **/
     int64 serverenCode = 5;
     /**变更数据的执行时间**/
     int64 executeTime = 6;
     /**变更数据的来源 **/
     oneof sourceType_present{
       Type sourceType = 7;
     }
   
     /**变更数据的schemaname **/
     string schemaName = 8;
   
     /**变更数据的tablename **/
     string tableName = 9;
     /**传输数据的事件长度**/
     int64 eventLength = 10;
     /**数据变更类型 **/
     oneof eventType_present{
       EventType eventType = 11;
     }
   
     /**预留扩展**/
     repeated Pair					props				= 12;
   
     /**当前事务的gitd**/
     string                 gtid                = 13;
   
     /**上游需要监听的数据源类型 **/
    enum Type{
      TYPECOMPATIBLEPROTO2 = 0;
      ORACLE = 1;
      MYSQL = 2;
      PGSQL = 3;
    }
   }
   ```


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] ChangjiGuo edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ChangjiGuo edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-895924402


   Hi, @maosuhan thanks a lot for your work. I found a bug with this pr.
   proto shcema: `map<uint32, bytes> pv_raw_features = 5;`
   generated code: 
   ```java
   Object elementDataVar5 = null;
   Map<Integer,ByteString> pbMap6 = message0.getPvRawFeaturesMap();;
   Map resultDataMap6 = new HashMap();
   for(Map.Entry<Integer,ByteString> pbEntry6: pbMap6.entrySet()){
   Object keyDataVar6= null;
   Object valueDataVar6= null;
   keyDataVar6 = ((Integer)pbEntry6.getKey());
   valueDataVar6 = ((ByteString)pbEntry6.getValue()).toByteArray();
   resultDataMap6.put(keyDataVar6, valueDataVar6);
   }
   elementDataVar5 = new GenericMapData(resultDataMap6);
   ```
   
   class compile error: Cannot determine simple type name "ByteString".
   
   My fix is ​​to manually import the ByteString class, and then it can be compiled successfully.
   `codegenAppender.appendLine("import " + ByteString.class.getName());`
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * c88b36dce672e7981125ac1267949a21fe7c2b7f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902) 
   * 076885377a4766f739fefa258da0b75c98e7a64a 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



[GitHub] [flink] ChangjiGuo commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ChangjiGuo commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-895924402


   Hi, @maosuhan thanks a lot for your work. I find a bug with this pr.
   proto shcema: `map<uint32, bytes> pv_raw_features = 5;`
   generated code: 
   ```java
   Object elementDataVar5 = null;
   Map<Integer,ByteString> pbMap6 = message0.getPvRawFeaturesMap();;
   Map resultDataMap6 = new HashMap();
   for(Map.Entry<Integer,ByteString> pbEntry6: pbMap6.entrySet()){
   Object keyDataVar6= null;
   Object valueDataVar6= null;
   keyDataVar6 = ((Integer)pbEntry6.getKey());
   valueDataVar6 = ((ByteString)pbEntry6.getValue()).toByteArray();
   resultDataMap6.put(keyDataVar6, valueDataVar6);
   }
   elementDataVar5 = new GenericMapData(resultDataMap6);
   ```
   
   error message: Cannot determine simple type name "ByteString".
   
   My fix is ​​to manually import the ByteString class, and then it can be compiled successfully.
   `codegenAppender.appendLine("import " + ByteString.class.getName());`
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-806273548


   @wuchong Thanks for your opinion.
   1. Regarding your suggestion, should we use `connector.<field_name>.default_value="<default_value>"`? The field type must be array or map because only these 2 types do not tolerate null values in protobuf. The `<default_value>` only support simple type like int/string/float/enum etc. And if user do not set this param, we will use protobuf's default value.  
   2. I have a look at the `org.apache.flink.table.data.conversion.StructuredObjectConverter#generateCode` and the implementation is similar to my code. And I can use `org.apache.flink.table.runtime.generated.CompileUtils` to finish code compiling and classloading work.
   
   Does it sound good? @wuchong @libenchao 


-- 
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



[GitHub] [flink] zhuzhengjun01 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
zhuzhengjun01 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1016148079


   @maosuhan hi, flink-protobuf works very well in my job, but i fount some corner case that it was not supported `oneof` in proto3.  so this will be dealt in future or not ?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1065194893


   @MartijnVisser hi, since 1.15 is released, can we move this PR forward  and have someone review the code?


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1065215659


   You're on top of the list, unfortunately 1.15 is not released yet but should happen rather soon


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-876480230


   Hi @maosuhan is there any update that you could let us know with regards to this PR? 


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] laughingman7743 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-858462749


   Is there any progress on this pull request?
   This pull request has been done very cool. I am hoping that Flink will support the ProtoBuf format.


-- 
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



[GitHub] [flink] sv3ndk commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
sv3ndk commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-814650640


   Thanks for the feed-back @maosuhan ,
   
   Indeed, I see what you mean, I think you're correct and I'm missing one case.
   
   If I understand things correctly, the logic we want is, when deserializing a pb scalar field into a Fink row field:
   
   1. pb2:
      1.a. if the pb scalar field is specified => set the Flink row field to that value
      1.b. if the pb scalar field is not specified but has a default value => set the Flink row field to that default
      1.c. if the pb scalar field is not specified, has no default value and is optional => set the Flink row field to null
      1.d. if the pb scalar field is not specified, has no default value and is not optional => set the Flink row field to JVM default
   2. pb3:
      2.a. if the scalar pb field is specifed => set the Flink row field to that value
      2.b. (not applicable: there is no equivalent of 1.b in pb3)
      2.c. (not applicable: scalar field are never optional in pb3 since the java generated class always returns a value and no `hasXYZ()` is present in that case)
      2.d. if the pb field is not specified=> set the Flink row field to JVM default
   
   This logic is, I think, the closest we can get to the way the generated classes themselves behaves, s.t. most of it should be achievable with as little logic as possible in the Flink format, by simply delegating to the generated classes. I admit using "JVM default" is surprising, although that's how protobuf generated classes behaves. My understanding is that the author of a protobuf schema would never use a non optional scalar field for communicating a nullable value, precisely because of that behavior.
   
   At the moment, I think the code in my MR supports all those cases except 1.c (I just wrote a UT for that case and indeed it fails). That case has no equivalent in "pure protobuf 2" since in that case the generated java class contains a primitive field (which cannot be null), although as you pointed out I should use `hasXYZ()` in order to set null in the Flink row.
   
   Here is what I suggest:
   
   * I add a couple of UT that make explicit that omission in my code 
   * I fix the bug for case 1.c, by adding a condition in `isMessageNonEmptyStr()` 
   * I regroup the UT into `proto2` and `proto3` folders s.t. it's easier to see which cases are covered in which versions
   * If it's ok for you, I suggest I remove the `readDefaultValues` flag, and we apply the logic I listed above in all cases, in order to keep the behavior as close as possible to the nominal protobuf behavior
   
   I'll wait for your feed-back before coding anything.


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * e3f953d91778ef96ad5fe222dd9da229dc2c5619 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323) 
   * 85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f 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



[GitHub] [flink] jianyun8023 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
jianyun8023 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-786349569


   We are supporting protobuf in the pulsar connector and expect this pr to be merged.
   Many thanks to the author.


----------------------------------------------------------------
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



[GitHub] [flink] libenchao commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
libenchao commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r595734073



##########
File path: flink-formats/flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
##########
@@ -0,0 +1 @@
+org.apache.flink.formats.protobuf.PbFormatFactory

Review comment:
       Add license header to this file.

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.PbSchemaValidator;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Deserialization schema from Protobuf to Flink types.
+ *
+ * <p>Deserializes a <code>byte[]</code> message as a protobuf object and reads the specified
+ * fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ */
+@PublicEvolving

Review comment:
       I think this is only for internal usage, not a public API?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.PbSchemaValidator;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Deserialization schema from Protobuf to Flink types.
+ *
+ * <p>Deserializes a <code>byte[]</code> message as a protobuf object and reads the specified
+ * fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ */
+@PublicEvolving
+public class PbRowDataDeserializationSchema implements DeserializationSchema<RowData> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PbRowDataDeserializationSchema.class);
+    private static final long serialVersionUID = -4040917522067315718L;
+
+    private final RowType rowType;
+    private final TypeInformation<RowData> resultTypeInfo;
+
+    private final String messageClassName;
+    private final boolean ignoreParseErrors;
+    private final boolean readDefaultValues;
+
+    private transient ProtoToRowConverter protoToRowConverter;
+
+    public PbRowDataDeserializationSchema(
+            RowType rowType,
+            TypeInformation<RowData> resultTypeInfo,
+            String messageClassName,
+            boolean ignoreParseErrors,
+            boolean readDefaultValues) {
+        checkNotNull(rowType, "Type information");
+        this.rowType = rowType;
+        this.resultTypeInfo = resultTypeInfo;
+        this.messageClassName = messageClassName;
+        this.ignoreParseErrors = ignoreParseErrors;
+        this.readDefaultValues = readDefaultValues;
+        // do it in client side to report error in the first place
+        new PbSchemaValidator(PbFormatUtils.getDescriptor(messageClassName), rowType).validate();
+        // this step is only used to validate codegen in client side in the first place
+        try {
+            // validate converter in client side to early detect errors
+            protoToRowConverter =
+                    new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
+        } catch (PbCodegenException e) {
+            throw new FlinkRuntimeException(e);
+        }
+    }
+
+    @Override
+    public void open(InitializationContext context) throws Exception {
+        protoToRowConverter = new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
+    }
+
+    @Override
+    public RowData deserialize(byte[] message) throws IOException {
+        try {
+            return protoToRowConverter.convertProtoBinaryToRow(message);
+        } catch (Throwable t) {
+            if (ignoreParseErrors) {
+                return null;
+            }
+            LOG.error("Failed to deserialize PB object.", t);

Review comment:
       we are throwing the error to the outside, hence we don't need to log it here.

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+    public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+        this.descriptor = descriptor;
+        this.rowType = rowType;
+        typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+        typeMatchMap.put(
+                JavaType.BYTE_STRING,
+                Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+        typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+        typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+        typeMatchMap.put(
+                JavaType.ENUM, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(
+                JavaType.STRING, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+        typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+    }
+
+    public Descriptors.Descriptor getDescriptor() {
+        return descriptor;
+    }
+
+    public void setDescriptor(Descriptors.Descriptor descriptor) {
+        this.descriptor = descriptor;
+    }
+
+    public RowType getRowType() {
+        return rowType;
+    }
+
+    public void setRowType(RowType rowType) {
+        this.rowType = rowType;
+    }
+
+    public void validate() {
+        validateTypeMatch(descriptor, rowType);
+        if (!descriptor
+                .getFile()
+                .getOptions()
+                .getJavaPackage()
+                .equals(descriptor.getFile().getPackage())) {
+            throw new IllegalArgumentException(
+                    "java_package and package must be the same in proto definition");
+        }
+        if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       why do you need this limitation?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenSimpleSerializer.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+/** Serializer to convert flink simple type data to proto simple type object. */
+public class PbCodegenSimpleSerializer implements PbCodegenSerializer {
+    private Descriptors.FieldDescriptor fd;
+    private LogicalType type;

Review comment:
       `final`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbRowTypeInformation.java
##########
@@ -0,0 +1,98 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+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.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+/** Generate Row type information according to pb descriptors. */
+public class PbRowTypeInformation {

Review comment:
       This is a util class, however the class name `PbRowTypeInformation` sounds like a subclass of `TypeInformation`, could we name it `PbRowTypeInformationUtil` or `PbRowTypeInformationGenerator`?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+    public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+        this.descriptor = descriptor;
+        this.rowType = rowType;
+        typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));

Review comment:
       we could move these initializations into a `static` block

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+    public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+        this.descriptor = descriptor;
+        this.rowType = rowType;
+        typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+        typeMatchMap.put(
+                JavaType.BYTE_STRING,
+                Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+        typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+        typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+        typeMatchMap.put(
+                JavaType.ENUM, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(
+                JavaType.STRING, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+        typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+    }
+
+    public Descriptors.Descriptor getDescriptor() {
+        return descriptor;
+    }
+
+    public void setDescriptor(Descriptors.Descriptor descriptor) {
+        this.descriptor = descriptor;
+    }
+
+    public RowType getRowType() {
+        return rowType;
+    }
+
+    public void setRowType(RowType rowType) {
+        this.rowType = rowType;
+    }
+
+    public void validate() {
+        validateTypeMatch(descriptor, rowType);
+        if (!descriptor
+                .getFile()
+                .getOptions()
+                .getJavaPackage()
+                .equals(descriptor.getFile().getPackage())) {
+            throw new IllegalArgumentException(
+                    "java_package and package must be the same in proto definition");
+        }
+        if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {
+            throw new IllegalArgumentException("java_multiple_files must set to true");
+        }
+    }
+
+    /**
+     * Validate type match of row type.
+     *
+     * @param descriptor

Review comment:
       better to give a description for the params.

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.PbSchemaValidator;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Deserialization schema from Protobuf to Flink types.
+ *
+ * <p>Deserializes a <code>byte[]</code> message as a protobuf object and reads the specified
+ * fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ */
+@PublicEvolving
+public class PbRowDataDeserializationSchema implements DeserializationSchema<RowData> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PbRowDataDeserializationSchema.class);
+    private static final long serialVersionUID = -4040917522067315718L;
+
+    private final RowType rowType;
+    private final TypeInformation<RowData> resultTypeInfo;
+
+    private final String messageClassName;
+    private final boolean ignoreParseErrors;
+    private final boolean readDefaultValues;
+
+    private transient ProtoToRowConverter protoToRowConverter;
+
+    public PbRowDataDeserializationSchema(
+            RowType rowType,
+            TypeInformation<RowData> resultTypeInfo,
+            String messageClassName,
+            boolean ignoreParseErrors,
+            boolean readDefaultValues) {
+        checkNotNull(rowType, "Type information");

Review comment:
       give a better error message

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbRowDataDeserializationSchema.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.PbSchemaValidator;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Deserialization schema from Protobuf to Flink types.
+ *
+ * <p>Deserializes a <code>byte[]</code> message as a protobuf object and reads the specified
+ * fields.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped IOExceptions.
+ */
+@PublicEvolving
+public class PbRowDataDeserializationSchema implements DeserializationSchema<RowData> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PbRowDataDeserializationSchema.class);
+    private static final long serialVersionUID = -4040917522067315718L;

Review comment:
       We suggest to use `1L` for `serialVersionUID`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    // protobuf code has a bug that, f_abc_7d will be convert to fAbc7d, but actually we need fAbc7D
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                // This closely matches the logic for ASCII characters in:
+                // http://google3/google/protobuf/descriptor.cc?l=249-251&rcl=228891689

Review comment:
       The link is broken?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();

Review comment:
       Maybe we could use a `EnumSet`  instead of `List<LogicalTypeRoot>`?  
   A step further, maybe a `EnumMap<JavaType, EnumSet<LogicalTypeRoot>>` is ok?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbFormatUtils.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.types.logical.LogicalType;
+
+import com.google.protobuf.Descriptors;
+
+/** Protobuf function util. */
+public class PbFormatUtils {
+
+    // protobuf code has a bug that, f_abc_7d will be convert to fAbc7d, but actually we need fAbc7D
+    public static String fieldNameToJsonName(String name) {
+        final int length = name.length();
+        StringBuilder result = new StringBuilder(length);
+        boolean isNextUpperCase = false;
+        for (int i = 0; i < length; i++) {
+            char ch = name.charAt(i);
+            if (ch == '_') {
+                isNextUpperCase = true;
+            } else if (isNextUpperCase) {
+                // This closely matches the logic for ASCII characters in:
+                // http://google3/google/protobuf/descriptor.cc?l=249-251&rcl=228891689
+                if ('a' <= ch && ch <= 'z') {
+                    ch = (char) (ch - 'a' + 'A');
+                    isNextUpperCase = false;
+                }
+                result.append(ch);
+            } else {
+                result.append(ch);
+            }
+        }
+        return result.toString();
+    }
+
+    public static boolean isSimpleType(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case BOOLEAN:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+            case BINARY:
+            case VARBINARY:
+                return true;
+            default:
+                return false;
+        }
+    }
+
+    public static String getStrongCamelCaseJsonName(String name) {
+        String jsonName = fieldNameToJsonName(name);
+        if (jsonName.length() == 1) {
+            return jsonName.toUpperCase();
+        } else {
+            return jsonName.substring(0, 1).toUpperCase() + jsonName.substring(1);
+        }
+    }
+
+    public static Descriptors.Descriptor getDescriptor(String className) {
+        try {
+            Class<?> pbClass = Class.forName(className);

Review comment:
       we should use the user classloader?
   CC @wuchong 

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbDecodingFormat.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.protobuf.deserialize.PbRowDataDeserializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.DecodingFormat;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+/** {@link DecodingFormat} for protobuf decoding. */
+public class PbDecodingFormat implements DecodingFormat<DeserializationSchema<RowData>> {
+    private String messageClassName;
+    private boolean ignoreParseErrors;
+    private boolean readDefaultValues;

Review comment:
       make these fields `final`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbRowDataSerializationSchema.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.PbSchemaValidator;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Serialization schema from Flink to Protobuf types.
+ *
+ * <p>Serializes a {@link RowData } to protobuf binary data.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped {@link FlinkRuntimeException}.
+ */
+@PublicEvolving

Review comment:
       the same with `PbRowDataDeserializationSchema`, I don't think this is a public API.

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+    public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+        this.descriptor = descriptor;
+        this.rowType = rowType;
+        typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+        typeMatchMap.put(
+                JavaType.BYTE_STRING,
+                Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+        typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+        typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+        typeMatchMap.put(
+                JavaType.ENUM, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(
+                JavaType.STRING, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+        typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+    }
+
+    public Descriptors.Descriptor getDescriptor() {
+        return descriptor;
+    }
+
+    public void setDescriptor(Descriptors.Descriptor descriptor) {
+        this.descriptor = descriptor;
+    }
+
+    public RowType getRowType() {
+        return rowType;
+    }
+
+    public void setRowType(RowType rowType) {
+        this.rowType = rowType;
+    }
+
+    public void validate() {
+        validateTypeMatch(descriptor, rowType);
+        if (!descriptor
+                .getFile()
+                .getOptions()
+                .getJavaPackage()
+                .equals(descriptor.getFile().getPackage())) {
+            throw new IllegalArgumentException(
+                    "java_package and package must be the same in proto definition");
+        }
+        if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {
+            throw new IllegalArgumentException("java_multiple_files must set to true");
+        }
+    }
+
+    /**
+     * Validate type match of row type.
+     *
+     * @param descriptor
+     * @param rowType
+     */
+    public void validateTypeMatch(Descriptors.Descriptor descriptor, RowType rowType) {
+        rowType.getFields()
+                .forEach(
+                        rowField -> {
+                            FieldDescriptor fieldDescriptor =
+                                    descriptor.findFieldByName(rowField.getName());
+                            if (null != fieldDescriptor) {
+                                validateTypeMatch(fieldDescriptor, rowField.getType());
+                            } else {
+                                throw new ValidationException(
+                                        "Column "
+                                                + rowField.getName()
+                                                + " does not exists in definition of proto class.");
+                            }
+                        });
+    }
+
+    /**
+     * Validate type match of general type.
+     *
+     * @param fd
+     * @param logicalType
+     */
+    public void validateTypeMatch(FieldDescriptor fd, LogicalType logicalType) {
+        if (!fd.isRepeated()) {
+            if (fd.getJavaType() != JavaType.MESSAGE) {
+                // simple type
+                validateSimpleType(fd, logicalType.getTypeRoot());
+            } else {
+                // message type
+                validateTypeMatch(fd.getMessageType(), (RowType) logicalType);

Review comment:
       if `logicalType` is not a `RowType`, then we'll get a `CastException`, maybe we can check it before casting to give a better message?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenRowDeserializer.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors.Descriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+
+/** Deserializer to convert proto message type object to flink row type data. */
+public class PbCodegenRowDeserializer implements PbCodegenDeserializer {
+    private Descriptor descriptor;
+    private RowType rowType;
+    private boolean readDefaultValues;
+    private PbCodegenAppender appender = new PbCodegenAppender();
+
+    public PbCodegenRowDeserializer(
+            Descriptor descriptor, RowType rowType, boolean readDefaultValues) {
+        this.rowType = rowType;
+        this.descriptor = descriptor;
+        this.readDefaultValues = readDefaultValues;
+    }
+
+    @Override
+    public String codegen(String returnInternalDataVarName, String pbGetStr)
+            throws PbCodegenException {
+        // The type of messageGetStr is a native pb object,
+        // it should be converted to RowData of flink internal type
+        PbCodegenVarId varUid = PbCodegenVarId.getInstance();
+        int uid = varUid.getAndIncrement();
+        String pbMessageVar = "message" + uid;
+        String rowDataVar = "rowData" + uid;
+
+        int fieldSize = rowType.getFieldNames().size();
+        String pbMessageTypeStr = PbFormatUtils.getFullJavaName(descriptor);
+        appender.appendLine(pbMessageTypeStr + " " + pbMessageVar + " = " + pbGetStr);
+        appender.appendLine(
+                "GenericRowData " + rowDataVar + " = new GenericRowData(" + fieldSize + ")");
+        int index = 0;
+        for (String fieldName : rowType.getFieldNames()) {
+            int subUid = varUid.getAndIncrement();
+            String elementDataVar = "elementDataVar" + subUid;
+
+            LogicalType subType = rowType.getTypeAt(rowType.getFieldIndex(fieldName));
+            FieldDescriptor elementFd = descriptor.findFieldByName(fieldName);
+            String strongCamelFieldName = PbFormatUtils.getStrongCamelCaseJsonName(fieldName);
+            PbCodegenDeserializer codegen =
+                    PbCodegenDeserializeFactory.getPbCodegenDes(
+                            elementFd, subType, readDefaultValues);
+            appender.appendLine("Object " + elementDataVar + " = null");
+            if (!readDefaultValues) {
+                // only works in syntax=proto2 and readDefaultValues=false
+                // readDefaultValues must be true in pb3 mode
+                String isMessageNonEmptyStr =
+                        isMessageNonEmptyStr(pbMessageVar, strongCamelFieldName, elementFd);
+                appender.appendSegment("if(" + isMessageNonEmptyStr + "){");
+            }
+            String elementMessageGetStr =
+                    pbMessageElementGetStr(pbMessageVar, strongCamelFieldName, elementFd);
+            if (!elementFd.isRepeated()) {
+                // field is not map or array
+                // this step is needed to convert primitive type to boxed type to unify the object

Review comment:
       Is this step really necessary? I could pass all the tests after removing this logic.

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbEncodingFormat.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.formats.protobuf.serialize.PbRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.format.EncodingFormat;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+/** {@link EncodingFormat} for protobuf encoding. */
+public class PbEncodingFormat implements EncodingFormat<SerializationSchema<RowData>> {
+    private String messageClassName;

Review comment:
       `final`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.AbstractMessage;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import org.codehaus.janino.ScriptEvaluator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * {@link RowToProtoConverter} can convert flink row data to binary protobuf message data by codegen
+ * process.
+ */
+public class RowToProtoConverter {
+    private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+    private ScriptEvaluator se;

Review comment:
       `final`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbRowDataSerializationSchema.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.PbSchemaValidator;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Serialization schema from Flink to Protobuf types.
+ *
+ * <p>Serializes a {@link RowData } to protobuf binary data.
+ *
+ * <p>Failures during deserialization are forwarded as wrapped {@link FlinkRuntimeException}.
+ */
+@PublicEvolving
+public class PbRowDataSerializationSchema implements SerializationSchema<RowData> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PbRowDataSerializationSchema.class);

Review comment:
       unused field?

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+
+import java.util.List;
+
+/** Serializer to convert flink row type data to proto row type object. */
+public class PbCodegenRowSerializer implements PbCodegenSerializer {
+    private List<Descriptors.FieldDescriptor> fds;
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;

Review comment:
       `final`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+    public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+        this.descriptor = descriptor;
+        this.rowType = rowType;
+        typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+        typeMatchMap.put(
+                JavaType.BYTE_STRING,
+                Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+        typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+        typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+        typeMatchMap.put(
+                JavaType.ENUM, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(
+                JavaType.STRING, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+        typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+    }
+
+    public Descriptors.Descriptor getDescriptor() {
+        return descriptor;
+    }
+
+    public void setDescriptor(Descriptors.Descriptor descriptor) {
+        this.descriptor = descriptor;
+    }
+
+    public RowType getRowType() {
+        return rowType;
+    }
+
+    public void setRowType(RowType rowType) {
+        this.rowType = rowType;
+    }
+
+    public void validate() {
+        validateTypeMatch(descriptor, rowType);
+        if (!descriptor
+                .getFile()
+                .getOptions()
+                .getJavaPackage()
+                .equals(descriptor.getFile().getPackage())) {
+            throw new IllegalArgumentException(
+                    "java_package and package must be the same in proto definition");
+        }
+        if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {
+            throw new IllegalArgumentException("java_multiple_files must set to true");
+        }
+    }
+
+    /**
+     * Validate type match of row type.
+     *
+     * @param descriptor
+     * @param rowType
+     */
+    public void validateTypeMatch(Descriptors.Descriptor descriptor, RowType rowType) {
+        rowType.getFields()
+                .forEach(
+                        rowField -> {
+                            FieldDescriptor fieldDescriptor =
+                                    descriptor.findFieldByName(rowField.getName());
+                            if (null != fieldDescriptor) {
+                                validateTypeMatch(fieldDescriptor, rowField.getType());
+                            } else {
+                                throw new ValidationException(
+                                        "Column "
+                                                + rowField.getName()
+                                                + " does not exists in definition of proto class.");
+                            }
+                        });
+    }
+
+    /**
+     * Validate type match of general type.
+     *
+     * @param fd
+     * @param logicalType
+     */
+    public void validateTypeMatch(FieldDescriptor fd, LogicalType logicalType) {
+        if (!fd.isRepeated()) {
+            if (fd.getJavaType() != JavaType.MESSAGE) {
+                // simple type
+                validateSimpleType(fd, logicalType.getTypeRoot());
+            } else {
+                // message type
+                validateTypeMatch(fd.getMessageType(), (RowType) logicalType);

Review comment:
       there are other type casts which may produce `CastException`, please add a check for them too.

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/ProtoToRowConverter.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.protobuf.deserialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryStringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FileDescriptor.Syntax;
+import org.codehaus.janino.ScriptEvaluator;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * {@link ProtoToRowConverter} can convert binary protobuf message data to flink row data by codegen
+ * process.
+ */
+public class ProtoToRowConverter {
+    private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+    private ScriptEvaluator se;
+    private Method parseFromMethod;

Review comment:
       make these fields `final`

##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/PbCodegenRowSerializer.java
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.PbCodegenVarId;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+
+import java.util.List;
+
+/** Serializer to convert flink row type data to proto row type object. */
+public class PbCodegenRowSerializer implements PbCodegenSerializer {
+    private List<Descriptors.FieldDescriptor> fds;

Review comment:
       unused field?




----------------------------------------------------------------
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



[GitHub] [flink] laughingman7743 commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r649208284



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.AbstractMessage;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * {@link RowToProtoConverter} can convert flink row data to binary protobuf message data by codegen
+ * process.
+ */
+public class RowToProtoConverter {
+    private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+
+    private final Method encodeMethod;
+
+    public RowToProtoConverter(RowType rowType, PbFormatConfig formatConfig)
+            throws PbCodegenException {
+        try {
+            Descriptors.Descriptor descriptor =
+                    PbFormatUtils.getDescriptor(formatConfig.getMessageClassName());
+            Class<?> messageClass = Class.forName(formatConfig.getMessageClassName());
+
+            PbCodegenAppender codegenAppender = new PbCodegenAppender();
+            String uuid = UUID.randomUUID().toString().replaceAll("\\-", "");
+            String generatedClassName = "GeneratedRowToProto_" + uuid;
+            String generatedPackageName = RowToProtoConverter.class.getPackage().getName();
+            codegenAppender.appendLine("package " + generatedPackageName);
+            codegenAppender.appendLine("import " + AbstractMessage.class.getName());
+            codegenAppender.appendLine("import " + Descriptors.class.getName());
+            codegenAppender.appendLine("import " + RowData.class.getName());
+            codegenAppender.appendLine("import " + ArrayData.class.getName());
+            codegenAppender.appendLine("import " + StringData.class.getName());
+            codegenAppender.appendLine("import " + ByteString.class.getName());
+            codegenAppender.appendLine("import " + List.class.getName());
+            codegenAppender.appendLine("import " + ArrayList.class.getName());
+            codegenAppender.appendLine("import " + Map.class.getName());
+            codegenAppender.appendLine("import " + HashMap.class.getName());
+
+            codegenAppender.appendSegment("public class " + generatedClassName + "{");
+            codegenAppender.appendSegment(
+                    "public static AbstractMessage "
+                            + PbConstant.GENERATED_ENCODE_METHOD
+                            + "(RowData rowData){");
+            codegenAppender.appendLine("AbstractMessage message = null");
+            PbCodegenSerializer codegenSer =
+                    PbCodegenSerializeFactory.getPbCodegenTopRowSer(
+                            descriptor, rowType, formatConfig);
+            String genCode = codegenSer.codegen("message", "rowData");
+            codegenAppender.appendSegment(genCode);
+            codegenAppender.appendLine("return message");
+            codegenAppender.appendSegment("}");
+            codegenAppender.appendSegment("}");
+
+            String printCode = codegenAppender.printWithLineNumber();
+            LOG.debug("Protobuf encode codegen: \n" + printCode);
+            Class generatedClass =
+                    PbCodegenUtils.compileClass(
+                            this.getClass().getClassLoader(),

Review comment:
       Does the class loader to be used here need to be a class loader that can be obtained from `getUserCodeClassLoader` in the `InitializationContext` argument of the `open` method of `PbRowDataSerializationSchema`? I think it is better to use `UserCodeClassLoader`.




-- 
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



[GitHub] [flink] ruanhang1993 commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-786494390


   Hi, @maosuhan thanks a lot for your work. I have used your code in flink 1.11.2. There seems to be some problems for me:
   
   1. The protobuf-java jar in flink-protobuf module conflicts with flink-dist module and flink-sql-connector-hive module.
   2. NullPointerException for field `protoToRowConverter` when recovering from some checkpoint
   
   For problem 1, I set the scope of the protobuf-java in flink-protobuf module to `provided`, and use the version setting by `protoc.version` in flink-parent. Besides, I relocated the protobuf-java in fink-sql-connector-hive.
   ```xml
   <dependency>
       <groupId>com.google.protobuf</groupId>
       <artifactId>protobuf-java</artifactId>
       <version>${protoc.version}</version>
       <scope>provided</scope>
   </dependency>
   ```
   
   For problem 2, it seems that the `open` method in `PbRowDeserializationSchema` is not called when recovering from some checkpoint, which causes the field `protoToRowConverter` to be null. I fixed it by the following code in `PbRowDeserializationSchema.java`.
   ```java
   @Override
   public RowData deserialize(byte[] message) throws IOException {
       try {
           if(protoToRowConverter == null) {
               LOG.info(String.format("[protobuf new]: %s %s %s", messageClassName, rowType.toString(), "" + readDefaultValues));
               protoToRowConverter = new ProtoToRowConverter(messageClassName, rowType, readDefaultValues);
           }
           return protoToRowConverter.convertProtoBinaryToRow(message);
       } catch (Throwable t) {
           if (ignoreParseErrors) {
               return null;
           }
           LOG.error("Failed to deserialize PB object.", t);
           throw new IOException("Failed to deserialize PB object.", t);
       }
   }
   ```
   
   Will the same problems occur in the new flink version ?
   And do you have any better ideas to fix it ?
   
   Thanks for reading.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744243277


   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 85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f (Fri May 28 11:05:33 UTC 2021)
   
   **Warnings:**
    * **3 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-18202).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r597519228



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** Validation class to verify protobuf definition and flink DDL schema. */
+public class PbSchemaValidator {
+    private Descriptors.Descriptor descriptor;
+    private RowType rowType;
+    private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+    public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+        this.descriptor = descriptor;
+        this.rowType = rowType;
+        typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+        typeMatchMap.put(
+                JavaType.BYTE_STRING,
+                Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+        typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+        typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+        typeMatchMap.put(
+                JavaType.ENUM, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(
+                JavaType.STRING, Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+        typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+        typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+    }
+
+    public Descriptors.Descriptor getDescriptor() {
+        return descriptor;
+    }
+
+    public void setDescriptor(Descriptors.Descriptor descriptor) {
+        this.descriptor = descriptor;
+    }
+
+    public RowType getRowType() {
+        return rowType;
+    }
+
+    public void setRowType(RowType rowType) {
+        this.rowType = rowType;
+    }
+
+    public void validate() {
+        validateTypeMatch(descriptor, rowType);
+        if (!descriptor
+                .getFile()
+                .getOptions()
+                .getJavaPackage()
+                .equals(descriptor.getFile().getPackage())) {
+            throw new IllegalArgumentException(
+                    "java_package and package must be the same in proto definition");
+        }
+        if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       I have removed the limitation now and user can set `java_outer_classname` and `java_multiple_files` freely.
   Also `java_package` and `package` can be set to different values.




-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * c88b36dce672e7981125ac1267949a21fe7c2b7f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bbfc0d96eb419932d49a54bf95f008c3155fbc81 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717) 
   
   <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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r611124468



##########
File path: flink-formats/flink-protobuf/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
##########
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */

Review comment:
       Thanks for your advice @laughingman7743. I have fixed the issue.




-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ec0895606f7b5cf68a6421be978b4e6fa30e838f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844) 
   * 6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b 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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r553095975



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/deserialize/PbCodegenSimpleDeserializer.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.protobuf.deserialize;
+
+import com.google.protobuf.Descriptors;
+
+public class PbCodegenSimpleDeserializer implements PbCodegenDeserializer {
+	private Descriptors.FieldDescriptor fd;
+
+	public PbCodegenSimpleDeserializer(
+		Descriptors.FieldDescriptor fd) {
+		this.fd = fd;
+	}
+
+	@Override
+	public String codegen(String returnVarName, String messageGetStr) {
+		StringBuilder sb = new StringBuilder();
+		switch (fd.getJavaType()) {
+			case INT:
+			case LONG:
+			case FLOAT:
+			case DOUBLE:
+			case BOOLEAN:
+				sb.append(returnVarName + " = " + messageGetStr + ";");
+				break;
+			case BYTE_STRING:
+				sb.append(returnVarName + " = " + messageGetStr + ".toByteArray();");
+				break;
+			case STRING:
+			case ENUM:
+				sb.append(
+					returnVarName + " = StringData.fromString(" + messageGetStr + ".toString());");

Review comment:
       Thanks for your patch. 




----------------------------------------------------------------
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



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-968345560


   @complone That would be great. Let me know if you have a PR that you would like to get reviewed or if you need any help


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-975009706


   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * ccf96fb549f1917fe888d69466a7e10013aa76ca Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-1019952384


   @zhuzhengjun01 hi, there's no way to distinguish between default and missing values in proto3 I think. At least there's no such API exposed in protobuf.
   And How do you reproduce the error of oneof, I can test successfully in my local environment.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-790666969


   @ruanhang1993 
   I revert the relocating change and protobuf version is aligned with flink parent project.
   ```
   		<dependency>
   			<groupId>com.google.protobuf</groupId>
   			<artifactId>protobuf-java</artifactId>
   			<version>${protoc.version}</version>
   		</dependency>
   ```
   I can successfully package flink project. Could you provide the package command when you see conflict?
   
   


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r553094997



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()
+			.equals(descriptor.getFile().getPackage())) {
+			throw new IllegalArgumentException(
+				"java_package and package must be the same in proto definition");
+		}
+		if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       It is the same issue of java_package/package, if java_multiple_files=false, the outer class name will also XXXOuter, also user can set java_outer_class_name to override the outer class name. Mixing with java_package/package, it is complicated to get the real java type of nested message type.




----------------------------------------------------------------
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



[GitHub] [flink] complone commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
complone commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-943001370


   I would like to ask how the support of this plugin is progressing, I am willing to carry out the remaining development work


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * d7b612acb72c25b4a30899155b764b6ad2535613 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612) 
   * 248aaeec8b1c42ca82610eebe34e2321d95f0320 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043) 
   
   <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



[GitHub] [flink] maosuhan edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-816481162


   @sv3ndk Thanks for your suggestion.
   It seems we can use `com.google.protobuf.Descriptors.FieldDescriptor.hasDefaultValue` to check whether the default value is set of a field.
   This proposal looks good to me. @libenchao @wuchong  Could you review his suggestion and give some feedback. If the proposal is accepted, I will work with @sv3ndk to merge his code to this PR. 


-- 
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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-755900070


   @wuchong , I have updated the code formatting in my PR.


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-816481162


   @sv3ndk Thanks for your suggestion.
   It seems we can use `com.google.protobuf.Descriptors.FieldDescriptor.hasDefaultValue` to check whether the default value is set of a field.
   This proposal looks good to me. Could you review @sv3ndk 's suggestion and give some feedback. If the proposal is accepted, I will work with @sv3ndk to merge his code to this PR. @libenchao @wuchong 


-- 
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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-828109372


   @sv3ndk Sorry for late reply.
   We had a discussion offline and we think that the rules that you proposed are more strict but introduce many different cases. We plan to deliver a more simple version of protobuf format in first release. So we will still use original 'read-default-values' options.


-- 
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



[GitHub] [flink] libenchao commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
libenchao commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-783940816


   @wuchong Sorry for the late reply, being a little recently. 
   I'll review this in next following two weeks.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4b702caaac7b53343401aee15f02454fd7a6f791 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MyLanPangzi commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MyLanPangzi commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-982321397


   @maosuhan  This would be a good option for users if it could be converted automatically. like this 
   PbCodegenSimpleDeserializer#codegen
   
   ```java
     if(type instanceof IntType){
                       sb.append(returnInternalDataVarName).append(" = ").append(pbGetStr).append(".getNumber();");
                       break;
                   }
                   sb.append(returnInternalDataVarName)
                       .append(" = BinaryStringData.fromString(").append(pbGetStr).append(".toString());");
                   break;
   
   ```
   
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-972994248


    @flinkbot


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0000",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * 0000 Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] syucream commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
syucream commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r555668256



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()
+			.equals(descriptor.getFile().getPackage())) {
+			throw new IllegalArgumentException(
+				"java_package and package must be the same in proto definition");
+		}
+		if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       For now I agree with the current implementation with the same reason of https://github.com/apache/flink/pull/14376#discussion_r555667701




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
wuchong commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-806354798


   I think we can simplify the desgin to just allow users change default value for types instead of columns. HBase connector also has a similar option `null-string-literal`. 
   
   https://ci.apache.org/projects/flink/flink-docs-master/docs/connectors/table/hbase/#null-string-literal


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 076885377a4766f739fefa258da0b75c98e7a64a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137) 
   * d7b612acb72c25b4a30899155b764b6ad2535613 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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347) 
   
   <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



[GitHub] [flink] complone edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
complone edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-943001370


   @wuchong This task does not seem to be progressing at the moment. May I assign this task to me? We currently have a need to support protobuf-format


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MyLanPangzi commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MyLanPangzi commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-963942917


   @maosuhan hello,i am testing the format in production. 
   source : kafka 
   format: pb3
   field count: 200+
   nested: 1
   qps: 40k/s
   partition: 200
   parallelism: 100
   sink: hive
   
   i found the consumer rate can't caught up.
   
   my log:
   spent: ms
   
   deserialize after : 1636447966334, spent: 0, count 97438 total spent 12322
   deserialize after : 1636447966335, spent: 0, count 96905 total spent 12065
   deserialize after : 1636447966341, spent: 0, count 99242 total spent 12850
   deserialize after : 1636447966342, spent: 0, count 97439 total spent 12322
   deserialize after : 1636447966344, spent: 0, count 96906 total spent 12065
   
   


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-972992579


   @maosuhan You can also trigger a new run via @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] MartijnVisser commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-973161506


   @maosuhan It probably makes sense to reduce the number of commits that are in this PR and also to rebase on the latest master branch. 


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * d73f43743b7343187b04f8d5a51c287dad1444a7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045) 
   * 7d049bb800be2ae6c53e63fd2917a514dab086dd 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 7d049bb800be2ae6c53e63fd2917a514dab086dd Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006) 
   
   <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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27985",
       "triggerID" : "a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * a3dd67481e55eec8a8d1a6dad219e1aa0ba10ec0 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27985) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765






-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 96a6ba2a83ca22091099ef36201afcbd8e719243 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794) 
   * 888452faa35e98b1151416484edfcf9e097f680f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 888452faa35e98b1151416484edfcf9e097f680f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96a6ba2a83ca22091099ef36201afcbd8e719243",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26794",
       "triggerID" : "975009706",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "888452faa35e98b1151416484edfcf9e097f680f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27741",
       "triggerID" : "888452faa35e98b1151416484edfcf9e097f680f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772",
       "triggerID" : "5cf0fa22d2ad2e24c9b4c088097999f2eff8f106",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 5cf0fa22d2ad2e24c9b4c088097999f2eff8f106 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=27772) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bbfc0d96eb419932d49a54bf95f008c3155fbc81 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717) 
   * 1c6f6ae6aec6f3bed305b56b6634422af8fd8da7 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



[GitHub] [flink] maosuhan edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-828337379


   @wuchong @libenchao 
   I have introduced a new connector param `write-null-string-literal`
   ```
   When serializing to protobuf data, this is the optional config to specify the string literal in protobuf's array/map in case of null values. By default empty string is used.
   Only string value in protobuf's complicated type can be specified. Numeric type will always pick 0 as default value.
   ```
   ## Extra improvements
   1. I use `SimpleCompiler` of janino to compile all my code which is the same way in `org.apache.flink.table.runtime.generated.CompileUtils`
   1. Add IT cases of source and sink in SQL.
   1. Add `write-null-string-literal`.


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a9a50e78d0ba6ff84d02fbadee1484970fac2c79 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709) 
   
   <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



[GitHub] [flink] laughingman7743 commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
laughingman7743 commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r652332960



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/serialize/RowToProtoConverter.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.protobuf.serialize;
+
+import org.apache.flink.formats.protobuf.PbCodegenAppender;
+import org.apache.flink.formats.protobuf.PbCodegenException;
+import org.apache.flink.formats.protobuf.PbCodegenUtils;
+import org.apache.flink.formats.protobuf.PbConstant;
+import org.apache.flink.formats.protobuf.PbFormatConfig;
+import org.apache.flink.formats.protobuf.PbFormatUtils;
+import org.apache.flink.formats.protobuf.deserialize.ProtoToRowConverter;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.AbstractMessage;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.Descriptors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * {@link RowToProtoConverter} can convert flink row data to binary protobuf message data by codegen
+ * process.
+ */
+public class RowToProtoConverter {
+    private static final Logger LOG = LoggerFactory.getLogger(ProtoToRowConverter.class);
+
+    private final Method encodeMethod;
+
+    public RowToProtoConverter(RowType rowType, PbFormatConfig formatConfig)
+            throws PbCodegenException {
+        try {
+            Descriptors.Descriptor descriptor =
+                    PbFormatUtils.getDescriptor(formatConfig.getMessageClassName());
+            Class<?> messageClass = Class.forName(formatConfig.getMessageClassName());
+
+            PbCodegenAppender codegenAppender = new PbCodegenAppender();
+            String uuid = UUID.randomUUID().toString().replaceAll("\\-", "");
+            String generatedClassName = "GeneratedRowToProto_" + uuid;
+            String generatedPackageName = RowToProtoConverter.class.getPackage().getName();
+            codegenAppender.appendLine("package " + generatedPackageName);
+            codegenAppender.appendLine("import " + AbstractMessage.class.getName());
+            codegenAppender.appendLine("import " + Descriptors.class.getName());
+            codegenAppender.appendLine("import " + RowData.class.getName());
+            codegenAppender.appendLine("import " + ArrayData.class.getName());
+            codegenAppender.appendLine("import " + StringData.class.getName());
+            codegenAppender.appendLine("import " + ByteString.class.getName());
+            codegenAppender.appendLine("import " + List.class.getName());
+            codegenAppender.appendLine("import " + ArrayList.class.getName());
+            codegenAppender.appendLine("import " + Map.class.getName());
+            codegenAppender.appendLine("import " + HashMap.class.getName());
+
+            codegenAppender.appendSegment("public class " + generatedClassName + "{");
+            codegenAppender.appendSegment(
+                    "public static AbstractMessage "
+                            + PbConstant.GENERATED_ENCODE_METHOD
+                            + "(RowData rowData){");
+            codegenAppender.appendLine("AbstractMessage message = null");
+            PbCodegenSerializer codegenSer =
+                    PbCodegenSerializeFactory.getPbCodegenTopRowSer(
+                            descriptor, rowType, formatConfig);
+            String genCode = codegenSer.codegen("message", "rowData");
+            codegenAppender.appendSegment(genCode);
+            codegenAppender.appendLine("return message");
+            codegenAppender.appendSegment("}");
+            codegenAppender.appendSegment("}");
+
+            String printCode = codegenAppender.printWithLineNumber();
+            LOG.debug("Protobuf encode codegen: \n" + printCode);
+            Class generatedClass =
+                    PbCodegenUtils.compileClass(
+                            this.getClass().getClassLoader(),

Review comment:
       I haven't experienced any particular errors with this ProtoBuf formatter, but I was curious about it because I have experienced NoClassDefFoundError and ClassNotFoundException related to Flink's class loader in the past.
   I'm not familiar with Flink's class loader. If the current implementation looks fine, I think it's fine to leave it as is.




-- 
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



[GitHub] [flink] ruanhang1993 edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
ruanhang1993 edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-791176342


   There is no problem when packaging flink project. The problem occurs when submitting job by flink cli. 
   I get the exception without the provided scope, like this. The test job aims to write data from kafka to hive.
   ```java
   Caused by: java.lang.ClassCastException: com.google.protobuf.Descriptors$Descriptor cannot be cast to com.google.protobuf.Descriptors$Descriptor
           at org.apache.flink.formats.protobuf.PbFormatUtils.getDescriptor(PbFormatUtils.java:81) ~[?:?]
          ......
   ```
   The flink directory `lib` contains the `flink-dist` jar(protobuf 3.11.1) and `flink-sql-connector-hive-1.2.2_2.11` jar(protobuf 2.5.0 relocated by me). The `flink-protobuf` jar(protobuf 3.11.1) is in my job jar. And submit job by this command:
   ```bash
   flink run  -m  yarn-cluster  \
   -yd  ...... -yt  protobufMessage.jar  \
   -c  package.Main  myJob.jar  jobParams \
   ```
   ---------------------------------------------------------------------------
   After a few tests, I think the problem is about the class loading in flink, not conflicting with other modules as I thought. 
   
   I need to place the `flink-protobuf` jar under the `lib` directory like other formats, e.g. `flink-json`. And all problems are gone. We don't need to change the version in `flink-protobuf` to `protoc.version` or relocate it in `flink-sql-connector-hive`.
   
   It seems that I use the jar in a wrong way. Thanks a lot for your answer.


----------------------------------------------------------------
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



[GitHub] [flink] maosuhan commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r553093799



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()

Review comment:
       This is a good question. If we omit java_package or set a different value with package, it is a little complicated to find out the real java type of nested message.
   For example,
        ```
        package foo.bar
   
        option java_package = "com.example.protos"
   
        message A{
   
             message B{}
   
        }
       ```
   When we get Descriptor of B, we cannot get the real java class name directly by using com.google.protobuf.Descriptors.Descriptor.getName(), the name is comming from package instead of java_package.
   So I just make it simple by forcibly set package the same with java_package.
   
   I tried to make it flexible to set different values, but it made the code a little complicated and not very compatible.
   
   Do you have any good ideas?
   
   




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c6f6ae6aec6f3bed305b56b6634422af8fd8da7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900) 
   * a721a9d5f663117a203628547a19b33b23927ac2 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



[GitHub] [flink] syucream commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
syucream commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r555667701



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()

Review comment:
       I got the background. That looks good to me. Additionally specifying `java_package` (and `java_multiple_files = true`) is not so unnatural, actually some google-provided .proto contains these options. https://github.com/protocolbuffers/protobuf/tree/master/src/google/protobuf




----------------------------------------------------------------
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



[GitHub] [flink] syucream commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
syucream commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-781216459


   Any progress?


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4b702caaac7b53343401aee15f02454fd7a6f791 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



[GitHub] [flink] wuchong commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
wuchong commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-805787270


   Regarding to the topics @libenchao  raised,
   
   1.  I perfer not failing the job. Filling with default values sounds good to me. But the default value should be configurable. 
   2. Generating code using Java is tricky now. There are some existing code as a references, e.g. `org.apache.flink.table.data.conversion.StructuredObjectConverter#generateCode`. 


-- 
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1c6f6ae6aec6f3bed305b56b6634422af8fd8da7 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900) 
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * c88b36dce672e7981125ac1267949a21fe7c2b7f 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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4b702caaac7b53343401aee15f02454fd7a6f791 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842) 
   * ec0895606f7b5cf68a6421be978b4e6fa30e838f 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



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-897519793


   @ChangjiGuo thanks for your comment and findings. I will fix this bug as you suggested.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] syucream commented on a change in pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
syucream commented on a change in pull request #14376:
URL: https://github.com/apache/flink/pull/14376#discussion_r547024452



##########
File path: flink-formats/flink-protobuf/src/main/java/org/apache/flink/formats/protobuf/PbSchemaValidator.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.protobuf;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.google.protobuf.Descriptors;
+import com.google.protobuf.Descriptors.FieldDescriptor;
+import com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class PbSchemaValidator {
+	private Descriptors.Descriptor descriptor;
+	private RowType rowType;
+	private Map<JavaType, List<LogicalTypeRoot>> typeMatchMap = new HashMap();
+
+	public PbSchemaValidator(Descriptors.Descriptor descriptor, RowType rowType) {
+		this.descriptor = descriptor;
+		this.rowType = rowType;
+		typeMatchMap.put(JavaType.BOOLEAN, Collections.singletonList(LogicalTypeRoot.BOOLEAN));
+		typeMatchMap.put(
+			JavaType.BYTE_STRING,
+			Arrays.asList(LogicalTypeRoot.BINARY, LogicalTypeRoot.VARBINARY));
+		typeMatchMap.put(JavaType.DOUBLE, Collections.singletonList(LogicalTypeRoot.DOUBLE));
+		typeMatchMap.put(JavaType.FLOAT, Collections.singletonList(LogicalTypeRoot.FLOAT));
+		typeMatchMap.put(
+			JavaType.ENUM,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(
+			JavaType.STRING,
+			Arrays.asList(LogicalTypeRoot.VARCHAR, LogicalTypeRoot.CHAR));
+		typeMatchMap.put(JavaType.INT, Collections.singletonList(LogicalTypeRoot.INTEGER));
+		typeMatchMap.put(JavaType.LONG, Collections.singletonList(LogicalTypeRoot.BIGINT));
+	}
+
+	public Descriptors.Descriptor getDescriptor() {
+		return descriptor;
+	}
+
+	public void setDescriptor(Descriptors.Descriptor descriptor) {
+		this.descriptor = descriptor;
+	}
+
+	public RowType getRowType() {
+		return rowType;
+	}
+
+	public void setRowType(RowType rowType) {
+		this.rowType = rowType;
+	}
+
+	public void validate() {
+		validateTypeMatch(descriptor, rowType);
+		if (!descriptor.getFile().getOptions().getJavaPackage()
+			.equals(descriptor.getFile().getPackage())) {
+			throw new IllegalArgumentException(
+				"java_package and package must be the same in proto definition");
+		}
+		if (!descriptor.getFile().getOptions().getJavaMultipleFiles()) {

Review comment:
       `java_multiple_files` is a java option. Can we support the case with `java_multiple_files = false`?




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789) 
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "",
       "status" : "CANCELED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   *  Unknown: [CANCELED](TBD) 
   * 38147f38be0a942edacc21b668aad7ea4e8f2c16 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] maosuhan commented on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
maosuhan commented on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-974770663


   @flinkbot run azure


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   * ccf96fb549f1917fe888d69466a7e10013aa76ca UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26717",
       "triggerID" : "ccf96fb549f1917fe888d69466a7e10013aa76ca",
       "triggerType" : "PUSH"
     }, {
       "hash" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26734",
       "triggerID" : "12bc9ca2640c7773b2ca7fa50e204605183ff309",
       "triggerType" : "PUSH"
     }, {
       "hash" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26789",
       "triggerID" : "38147f38be0a942edacc21b668aad7ea4e8f2c16",
       "triggerType" : "PUSH"
     }, {
       "hash" : "",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "974770663",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790",
       "triggerID" : "f5ab7477f53088b7f6794e113fdb4be6248ad230",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * f5ab7477f53088b7f6794e113fdb4be6248ad230 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26790) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #14376: [FLINK-18202][PB format] New Format of protobuf

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #14376:
URL: https://github.com/apache/flink/pull/14376#issuecomment-744252765


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10842",
       "triggerID" : "4b702caaac7b53343401aee15f02454fd7a6f791",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10844",
       "triggerID" : "ec0895606f7b5cf68a6421be978b4e6fa30e838f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10860",
       "triggerID" : "6b2d6e05fdbd4b579c68007e5001bd1cabe1dd7b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11709",
       "triggerID" : "a9a50e78d0ba6ff84d02fbadee1484970fac2c79",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11717",
       "triggerID" : "bbfc0d96eb419932d49a54bf95f008c3155fbc81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13900",
       "triggerID" : "1c6f6ae6aec6f3bed305b56b6634422af8fd8da7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a721a9d5f663117a203628547a19b33b23927ac2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8f1051527a46aeba13aadc2db3c5f34d8aff78d2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13902",
       "triggerID" : "c88b36dce672e7981125ac1267949a21fe7c2b7f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14137",
       "triggerID" : "076885377a4766f739fefa258da0b75c98e7a64a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=14612",
       "triggerID" : "d7b612acb72c25b4a30899155b764b6ad2535613",
       "triggerType" : "PUSH"
     }, {
       "hash" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=15043",
       "triggerID" : "248aaeec8b1c42ca82610eebe34e2321d95f0320",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=16330",
       "triggerID" : "bd27dabd13c11fd65b98423595e9d93234538444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17323",
       "triggerID" : "e3f953d91778ef96ad5fe222dd9da229dc2c5619",
       "triggerType" : "PUSH"
     }, {
       "hash" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=17347",
       "triggerID" : "85d79ad7a47d32626145bf3ff2a2ddf7bbb8575f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19010",
       "triggerID" : "058a86bf55557a420b625c0a96f2a4cf0a323fe3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=19045",
       "triggerID" : "d73f43743b7343187b04f8d5a51c287dad1444a7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=22006",
       "triggerID" : "7d049bb800be2ae6c53e63fd2917a514dab086dd",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "24361517f485cb00e0b85f6e201419da072334b5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972994248",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "24361517f485cb00e0b85f6e201419da072334b5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "0000",
       "status" : "DELETED",
       "url" : "TBD",
       "triggerID" : "972992579",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * a721a9d5f663117a203628547a19b33b23927ac2 UNKNOWN
   * 8f1051527a46aeba13aadc2db3c5f34d8aff78d2 UNKNOWN
   * 24361517f485cb00e0b85f6e201419da072334b5 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=26709) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org