[
https://issues.apache.org/jira/browse/FLINK-3702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15434750#comment-15434750
]
ASF GitHub Bot commented on FLINK-3702:
---------------------------------------
Github user ggevay commented on a diff in the pull request:
https://github.com/apache/flink/pull/2094#discussion_r76040516
--- Diff:
flink-core/src/main/java/org/apache/flink/api/common/typeinfo/TypeInformation.java
---
@@ -160,6 +161,53 @@ public boolean isSortKeyType() {
@PublicEvolving
public abstract TypeSerializer<T> createSerializer(ExecutionConfig
config);
+
+ /**
+ * Creates a {@link FieldAccessor} for the given field position, which
can be used to get and set
+ * the specified field on instances of this type.
+ *
+ * @param pos The field position (zero-based)
+ * @param config Configuration object
+ * @param <F> The type of the field to access
+ * @return The created FieldAccessor
+ */
+ @PublicEvolving
+ public <F> FieldAccessor<T, F> getFieldAccessor(int pos,
ExecutionConfig config){
+ throw new InvalidFieldReferenceException("Cannot reference
field by position on " + this.toString()
+ + "Referencing a field by position is supported on
tuples, case classes, and arrays. "
+ + "Additionally, you can select the 0th field of a
primitive/basic type (e.g. int).");
+ }
+
+ /**
+ * Creates a {@link FieldAccessor} for the field that is given by a
field expression,
+ * which can be used to get and set the specified field on instances of
this type.
+ *
+ * @see <a
href="https://ci.apache.org/projects/flink/flink-docs-master/apis/common/index.html#define-keys-using-field-expressions">
+ * Defining keys using Field Expressions</a>
+ *
+ * @param field The field expression
+ * @param config Configuration object
+ * @param <F> The type of the field to access
+ * @return The created FieldAccessor
+ */
+ @PublicEvolving
+ public <F> FieldAccessor<T, F> getFieldAccessor(String field,
ExecutionConfig config) {
+ throw new InvalidFieldReferenceException("Cannot reference
field by field expression on " + this.toString()
+ + "Field expressions are only supported on POJO
types, tuples, and case classes. "
+ + "For the requirements for a type to be
considered a POJO, see "
+ +
"https://ci.apache.org/projects/flink/flink-docs-master/apis/common/index.html#pojos");
--- End diff --
f0bdc99a33612b5326f134daf8f36e2c4553fe3a
> DataStream API PojoFieldAccessor doesn't support nested POJOs
> -------------------------------------------------------------
>
> Key: FLINK-3702
> URL: https://issues.apache.org/jira/browse/FLINK-3702
> Project: Flink
> Issue Type: Improvement
> Components: DataStream API
> Affects Versions: 1.0.0
> Reporter: Robert Metzger
> Assignee: Gabor Gevay
>
> The {{PojoFieldAccessor}} (which is used by {{.sum(String)}} and similar
> methods) doesn't support nested POJOs right now.
> As part of FLINK-3697 I'll add a check for a nested POJO and fail with an
> exception.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)