[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16646244#comment-16646244
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

klion26 closed pull request #6430: [FLINK-8058][Queryable State]Queryable state 
should check types
URL: https://github.com/apache/flink/pull/6430
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
 
b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
index 470c7acf9d7..70f2c31642c 100644
--- 
a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
+++ 
b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/QueryableStateClient.java
@@ -263,14 +263,16 @@ public ExecutionConfig setExecutionConfig(ExecutionConfig 
config) {
stateDescriptor.initializeSerializerUnlessSet(executionConfig);
 
final byte[] serializedKeyAndNamespace;
+   final byte[] serializedStateDescriptor;
try {
serializedKeyAndNamespace = KvStateSerializer
.serializeKeyAndNamespace(key, 
keySerializer, namespace, namespaceSerializer);
+   serializedStateDescriptor = 
KvStateSerializer.serializedStateDescriptor(stateDescriptor);
} catch (IOException e) {
return FutureUtils.getFailedFuture(e);
}
 
-   return getKvState(jobId, queryableStateName, key.hashCode(), 
serializedKeyAndNamespace)
+   return getKvState(jobId, queryableStateName, key.hashCode(), 
serializedKeyAndNamespace, serializedStateDescriptor)
.thenApply(stateResponse -> createState(stateResponse, 
stateDescriptor));
}
 
@@ -306,10 +308,12 @@ public ExecutionConfig setExecutionConfig(ExecutionConfig 
config) {
final JobID jobId,
final String queryableStateName,
final int keyHashCode,
-   final byte[] serializedKeyAndNamespace) {
+   final byte[] serializedKeyAndNamespace,
+   final byte[] serializedStateDescriptor) {
LOG.debug("Sending State Request to {}.", remoteAddress);
try {
-   KvStateRequest request = new KvStateRequest(jobId, 
queryableStateName, keyHashCode, serializedKeyAndNamespace);
+   KvStateRequest request = new KvStateRequest(jobId, 
queryableStateName, keyHashCode,
+   
serializedKeyAndNamespace, serializedStateDescriptor);
return client.sendRequest(remoteAddress, request);
} catch (Exception e) {
LOG.error("Unable to send KVStateRequest: ", e);
diff --git 
a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
 
b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
index 4a64678e550..0caf68b28a7 100644
--- 
a/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
+++ 
b/flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
@@ -18,12 +18,17 @@
 
 package org.apache.flink.queryablestate.client.state.serialization;
 
+import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputDeserializer;
 import org.apache.flink.core.memory.DataOutputSerializer;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -264,4 +269,39 @@
return null;
}
}
+
+   /**
+* Serialize a stateDescriptor to bytes[].
+* @param stateDescriptor the value will be 

[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16646243#comment-16646243
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

klion26 commented on issue #6430: [FLINK-8058][Queryable State]Queryable state 
should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-428901980
 
 
   @azagrebin ok, i'll close the PR.


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16646103#comment-16646103
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

azagrebin edited a comment on issue #6430: [FLINK-8058][Queryable 
State]Queryable state should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-428861922
 
 
   @klion26, I suggest we get back to Jira issue and discuss the approach. In 
general, if design is firstly described in Jira issue and there is a feedback 
from a committer, it simplifies and speeds up the review and merging. I would 
close this PR and open another one when there is an agreement in Jira issue 
about the implementation approach.
   cc @kl0u 


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16646101#comment-16646101
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

azagrebin commented on issue #6430: [FLINK-8058][Queryable State]Queryable 
state should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-428861922
 
 
   @klion26, I suggest we get back to Jira issue and discuss the approach. In 
general, if design is firstly described in Jira issue and there is a feedback 
from a committer, it simplifies and speeds up the review and merging. I would 
close this PR and open another one when there is an agreement in Jira issue 
about the implementation approach.


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-10-02 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16635178#comment-16635178
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

azagrebin commented on issue #6430: [FLINK-8058][Queryable State]Queryable 
state should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-426203590
 
 
   hi @klion26, do I understand correctly that the idea now is that the state 
descriptor is serialised and sent for every request? If so it is probably too 
expensive to do it every time. The check could happen only once.


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-08-06 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16570958#comment-16570958
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

klion26 commented on issue #6430: [FLINK-8058][Queryable State]Queryable state 
should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-410893513
 
 
   hi, @kl0u @twalthr Could you please help reviewing this?


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-27 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16560627#comment-16560627
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

klion26 commented on issue #6430: [FLINK-8058][Queryable State]Queryable state 
should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-408584575
 
 
   @yanghua thank you for helping review, I have updated the code~


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-27 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16559662#comment-16559662
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

yanghua commented on a change in pull request #6430: [FLINK-8058][Queryable 
State]Queryable state should check types
URL: https://github.com/apache/flink/pull/6430#discussion_r205752940
 
 

 ##
 File path: 
flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
 ##
 @@ -81,6 +83,14 @@ public KvStateServerHandler(
} else {
byte[] serializedKeyAndNamespace = 
request.getSerializedKeyAndNamespace();
 
+   StateDescriptor requestStateDescriptor = 
KvStateSerializer.deserializeStateDescriptor(request.getSerializedStateDescriptor());
+   StateDescriptor registStateDescriptor = 
kvState.getStateDescriptor();
+
+   
Preconditions.checkArgument(requestStateDescriptor.getType().equals(registStateDescriptor.getType()),
+   "State type mismatch, need[" + 
registStateDescriptor.getType() + "] gotten[" + 
requestStateDescriptor.getType() + "]");
 
 Review comment:
   use string format looks better~


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-27 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16559661#comment-16559661
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

yanghua commented on a change in pull request #6430: [FLINK-8058][Queryable 
State]Queryable state should check types
URL: https://github.com/apache/flink/pull/6430#discussion_r205753939
 
 

 ##
 File path: 
flink-queryable-state/flink-queryable-state-client-java/src/main/java/org/apache/flink/queryablestate/client/state/serialization/KvStateSerializer.java
 ##
 @@ -264,4 +269,39 @@
return null;
}
}
+
+   /**
+* Serialize a stateDescriptor to bytes[].
+* @param stateDescriptor the value will be serialized.
+*
+* @return The serialized values
+* @throws IOException On failure during serialization
+*/
+   public static byte[] serializedStateDescriptor(StateDescriptor 
stateDescriptor) throws IOException {
+   ByteArrayOutputStream bos = new ByteArrayOutputStream();
+   ObjectOutputStream out = new ObjectOutputStream(bos);
+   out.writeObject(stateDescriptor);
+   out.close();
 
 Review comment:
   shall we use `try-with-resource` to make sure the stream can be closed?


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-27 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16559663#comment-16559663
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

yanghua commented on a change in pull request #6430: [FLINK-8058][Queryable 
State]Queryable state should check types
URL: https://github.com/apache/flink/pull/6430#discussion_r205753009
 
 

 ##
 File path: 
flink-queryable-state/flink-queryable-state-runtime/src/main/java/org/apache/flink/queryablestate/server/KvStateServerHandler.java
 ##
 @@ -81,6 +83,14 @@ public KvStateServerHandler(
} else {
byte[] serializedKeyAndNamespace = 
request.getSerializedKeyAndNamespace();
 
+   StateDescriptor requestStateDescriptor = 
KvStateSerializer.deserializeStateDescriptor(request.getSerializedStateDescriptor());
+   StateDescriptor registStateDescriptor = 
kvState.getStateDescriptor();
+
+   
Preconditions.checkArgument(requestStateDescriptor.getType().equals(registStateDescriptor.getType()),
+   "State type mismatch, need[" + 
registStateDescriptor.getType() + "] gotten[" + 
requestStateDescriptor.getType() + "]");
+   
Preconditions.checkArgument(requestStateDescriptor.getSerializer().equals(registStateDescriptor.getSerializer()),
+   "State value serializer mismatch, need 
[" + registStateDescriptor.getSerializer() + "] gotten[" + 
requestStateDescriptor.getSerializer() + "]");
 
 Review comment:
   use string format looks better~


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Assignee: Congxian Qiu
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-26 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16558362#comment-16558362
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

klion26 commented on issue #6430: [FLINK-8058][Queryable State]Queryable state 
should check types
URL: https://github.com/apache/flink/pull/6430#issuecomment-408114155
 
 
   @kl0u @twalthr could you please help review this pr?
   In other word, I have one more question: If I need to add a test 
that`KvStateSerializer#deserializeStateDescriptor` throws 
`ClassNotFoundException`, if yes, if there any test I can refer.


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-26 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16558354#comment-16558354
 ] 

ASF GitHub Bot commented on FLINK-8058:
---

klion26 opened a new pull request #6430: [FLINK-8058][Queryable State]Queryable 
state should check types
URL: https://github.com/apache/flink/pull/6430
 
 
   ## What is the purpose of the change
   
   Add type checks(state type and state value type) in server side
   
   
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
 - *Added mismatch state type and state value type 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, Yarn/Mesos, ZooKeeper: no
 - The S3 file system connector: no
   
   ## Documentation
   
 - Does this pull request introduce a new feature?  no
   


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


> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Priority: Major
>  Labels: pull-request-available
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-21 Thread Congxian Qiu (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16551703#comment-16551703
 ] 

Congxian Qiu commented on FLINK-8058:
-

Hi, [~kkl0u], is this issues needed in Queryable State, if it is needed. I'm 
interested in it.

Could I check the state type and type of contained values in JobMaster or in 
KvStateServerHandler? I perfer adding the statedescriptor in JobMaster and 
check all the things when looking up state location, what about your opinion?

> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Priority: Major
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-19 Thread Chesnay Schepler (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16550229#comment-16550229
 ] 

Chesnay Schepler commented on FLINK-8058:
-

I don't know much about how queryable state works, this is just something i 
noticed while using it.
Maybe [~kkl0u] can  help you.

> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Priority: Major
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-19 Thread Congxian Qiu (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16550174#comment-16550174
 ] 

Congxian Qiu commented on FLINK-8058:
-

Hi, [~Zentol] could I check the state type and type of contained values in 
JobMaster or in KvStateServerHandler? I perfer adding the statedescriptor in 
Jobmaster and check all the things when looking up state location, what about 
your opinion? 

Looking forward to your reply.

> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Priority: Major
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-8058) Queryable state should check types

2018-07-17 Thread Congxian Qiu (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-8058?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16547316#comment-16547316
 ] 

Congxian Qiu commented on FLINK-8058:
-

Hi, [~Zentol] is this issues still needed? If yes, can I  do it.

> Queryable state should check types
> --
>
> Key: FLINK-8058
> URL: https://issues.apache.org/jira/browse/FLINK-8058
> Project: Flink
>  Issue Type: Improvement
>  Components: Queryable State
>Affects Versions: 1.4.0
>Reporter: Chesnay Schepler
>Priority: Major
>
> The queryable state currently doesn't do any type checks on the client or 
> server and generally relies on serializers to catch errors.
> Neither the type of state is checked (ValueState, ListState etc.) nor the 
> type of contained values.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)