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

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

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1905#discussion_r60400970
  
    --- Diff: 
flink-libraries/flink-cep-scala/src/main/scala/org/apache/flink/cep/scala/package.scala
 ---
    @@ -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.cep
    +
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.scala.ClosureCleaner
    +import org.apache.flink.cep.{PatternStream => JPatternStream}
    +
    +import _root_.scala.reflect.ClassTag
    +
    +package object scala {
    +
    +  /**
    +    * Utility method to wrap { @link org.apache.flink.cep.PatternStream}
    +    * for usage with the Scala API.
    +    *
    +    * @param javaPatternStream The underlying pattern stream from the Java 
API
    +    * @tparam T Type of the events
    +    * @return A pattern stream from the Scala API which wraps a pattern 
stream from the Java API
    +    */
    +  private[flink] def wrapPatternStream[T: TypeInformation : 
ClassTag](javaPatternStream: JPatternStream[T])
    +  : scala.PatternStream[T] = {
    +    javaPatternStream match {
    +      case p: JPatternStream[T] => PatternStream[T](p)
    +      case _ => null
    --- End diff --
    
    I think we should throw an `Exception` if we encounter a `null` value here. 
The reason is that the subsequent operations would throw a 
`NullPointerException` otherwise. Moreover, there is no good strategy how to 
continue with a `null` pattern stream.
    
    The most scalaesque way would be:
    
    ```
    Option(javaPatternStream) match {
      case Some(p) => PatternStream[T](p)
      case None => throw new Exception(...)
    }
    ```


> Scala API for CEP
> -----------------
>
>                 Key: FLINK-3708
>                 URL: https://issues.apache.org/jira/browse/FLINK-3708
>             Project: Flink
>          Issue Type: Improvement
>          Components: CEP
>    Affects Versions: 1.1.0
>            Reporter: Till Rohrmann
>            Assignee: Stefan Richter
>
> Currently, The CEP library does not support Scala case classes, because the 
> {{TypeExtractor}} cannot handle them. In order to support them, it would be 
> necessary to offer a Scala API for the CEP library.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to