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/10/06 10:24:53 UTC

[GitHub] [flink] aljoscha opened a new pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

aljoscha opened a new pull request #13545:
URL: https://github.com/apache/flink/pull/13545


   ## What is the purpose of the change
   
   We now point out what might be the most likely cause for the exception.
   In addition, we remove a lot of code duplication.
   
   ## Verifying this change
   
   I added a new test.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): no
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
     - The serializers: no
     - The runtime per-record code paths (performance sensitive): no
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: no
     - The S3 file system connector: no


----------------------------------------------------------------
This is an automated message from the 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 #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234",
       "triggerID" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0acc4e1b8b45545a92dff78a2e171218e84240ae",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7261",
       "triggerID" : "0acc4e1b8b45545a92dff78a2e171218e84240ae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b66350a46c67dd801c8cb92ad0a33e2ad2e2d759 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234) 
   * 0acc4e1b8b45545a92dff78a2e171218e84240ae Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7261) 
   
   <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 #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234",
       "triggerID" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b66350a46c67dd801c8cb92ad0a33e2ad2e2d759 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234) 
   
   <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] aljoscha commented on pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   Thanks for the reviews, @AHeise! I merged it as a method 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] aljoscha commented on a change in pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/UnexpectedStateHandleException.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
+
+/**
+ * Signals that an operation did not get the type of {@link StateObject} that was expected. This can
+ * mostly happen when a different {@link StateBackend} from the one that was used for taking a
+ * checkpoint/savepoint is used when restoring.
+ */
+public class UnexpectedStateHandleException extends IllegalStateException {

Review comment:
       I'd be happy to change it to a small method if you think it's more maintainable. I'l also not sure I like the additional exception.




----------------------------------------------------------------
This is an automated message from the 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 #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234",
       "triggerID" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0acc4e1b8b45545a92dff78a2e171218e84240ae",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0acc4e1b8b45545a92dff78a2e171218e84240ae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b66350a46c67dd801c8cb92ad0a33e2ad2e2d759 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234) 
   * 0acc4e1b8b45545a92dff78a2e171218e84240ae 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] aljoscha commented on a change in pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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



##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/state/UnexpectedStateHandleExceptionTest.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.state;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link UnexpectedStateHandleException}.
+ */
+public class UnexpectedStateHandleExceptionTest {

Review comment:
       I added these tests to check whether the somewhat longer string actually renders correctly. And I did find a formatting problem. 😅
   
   I don't want to add a more complicated ITCase now but someone could definitely do 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] aljoscha commented on a change in pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/UnexpectedStateHandleException.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
+
+/**
+ * Signals that an operation did not get the type of {@link StateObject} that was expected. This can
+ * mostly happen when a different {@link StateBackend} from the one that was used for taking a
+ * checkpoint/savepoint is used when restoring.
+ */
+public class UnexpectedStateHandleException extends IllegalStateException {
+
+	@SuppressWarnings("unchecked")
+	public UnexpectedStateHandleException(
+			Class<? extends StateObject> expectedStateHandleClass,
+			Class<? extends StateObject> actualStateHandleClass) {
+		this(new Class[] {expectedStateHandleClass}, actualStateHandleClass);
+	}
+
+	public UnexpectedStateHandleException(
+			Class<? extends StateObject>[] expectedStateHandleClasses,
+			Class<? extends StateObject> actualStateHandleClass) {

Review comment:
       I thought about that as well but decided to go with the order that JUnit `assertEquals()` uses. 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] flinkbot commented on pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b66350a46c67dd801c8cb92ad0a33e2ad2e2d759 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] AHeise commented on a change in pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/UnexpectedStateHandleException.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
+
+/**
+ * Signals that an operation did not get the type of {@link StateObject} that was expected. This can
+ * mostly happen when a different {@link StateBackend} from the one that was used for taking a
+ * checkpoint/savepoint is used when restoring.
+ */
+public class UnexpectedStateHandleException extends IllegalStateException {

Review comment:
       If we could find a good place for such a method, I'd prefer a method. But if not (like adding a new Util class), leave as is. The only place that I could think of is to add a static method to `RestoreOperation`.

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/state/UnexpectedStateHandleExceptionTest.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.state;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link UnexpectedStateHandleException}.
+ */
+public class UnexpectedStateHandleExceptionTest {

Review comment:
       Hm, I'm not sure what the value of the current tests are. If I add a linebreak to the message formatting at some point, I'd need to touch both tests. 
   
   But I also don't insist on the ITCase: Since you didn't change anything on the exception logic (just using a subclass), I'm fine to leave 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] AHeise commented on a change in pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/UnexpectedStateHandleException.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
+
+/**
+ * Signals that an operation did not get the type of {@link StateObject} that was expected. This can
+ * mostly happen when a different {@link StateBackend} from the one that was used for taking a
+ * checkpoint/savepoint is used when restoring.
+ */
+public class UnexpectedStateHandleException extends IllegalStateException {

Review comment:
       I guess the benefit of using an additional exception to avoid code duplication. I'm a bit torn if a small service method would not achieve the same but I guess it's okay like this.

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/state/UnexpectedStateHandleException.java
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.shaded.guava18.com.google.common.base.Joiner;
+
+/**
+ * Signals that an operation did not get the type of {@link StateObject} that was expected. This can
+ * mostly happen when a different {@link StateBackend} from the one that was used for taking a
+ * checkpoint/savepoint is used when restoring.
+ */
+public class UnexpectedStateHandleException extends IllegalStateException {
+
+	@SuppressWarnings("unchecked")
+	public UnexpectedStateHandleException(
+			Class<? extends StateObject> expectedStateHandleClass,
+			Class<? extends StateObject> actualStateHandleClass) {
+		this(new Class[] {expectedStateHandleClass}, actualStateHandleClass);
+	}
+
+	public UnexpectedStateHandleException(
+			Class<? extends StateObject>[] expectedStateHandleClasses,
+			Class<? extends StateObject> actualStateHandleClass) {

Review comment:
       Swap arguments and use ellipse to avoid 2 ctors?

##########
File path: flink-runtime/src/test/java/org/apache/flink/runtime/state/UnexpectedStateHandleExceptionTest.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.runtime.state;
+
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests for {@link UnexpectedStateHandleException}.
+ */
+public class UnexpectedStateHandleExceptionTest {

Review comment:
       Rather then these no-op tests, I'd rather see an ITCase where we actually save with rocksdb and read with file state backend.




----------------------------------------------------------------
This is an automated message from the 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 #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234",
       "triggerID" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b66350a46c67dd801c8cb92ad0a33e2ad2e2d759 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234) 
   
   <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] aljoscha closed pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

Posted by GitBox <gi...@apache.org>.
aljoscha closed pull request #13545:
URL: https://github.com/apache/flink/pull/13545


   


----------------------------------------------------------------
This is an automated message from the 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 #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7234",
       "triggerID" : "b66350a46c67dd801c8cb92ad0a33e2ad2e2d759",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0acc4e1b8b45545a92dff78a2e171218e84240ae",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7261",
       "triggerID" : "0acc4e1b8b45545a92dff78a2e171218e84240ae",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0acc4e1b8b45545a92dff78a2e171218e84240ae Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7261) 
   
   <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 commented on pull request #13545: [FLINK-19486] Make "Unexpected State Handle" Exception more helpful

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


   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 b66350a46c67dd801c8cb92ad0a33e2ad2e2d759 (Tue Oct 06 10:27:06 UTC 2020)
   
   **Warnings:**
    * 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-19486).** 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