You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@apex.apache.org by tushargosavi <gi...@git.apache.org> on 2016/04/22 15:15:14 UTC

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

GitHub user tushargosavi opened a pull request:

    https://github.com/apache/incubator-apex-core/pull/313

    APEXCORE-107 Support for adding modules in the DAG through property and json file

    Support for adding modules in the DAG property file and json file.
    Also changed module API to make module as an operator.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tushargosavi/incubator-apex-core APEXCORE-107

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-apex-core/pull/313.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #313
    
----
commit 42c0134d748d1b3d6240d4d94d13173367aba842
Author: Tushar R. Gosavi <tu...@gmail.com>
Date:   2016-04-22T12:54:35Z

    APEXCORE-107 Support for adding modules in the DAG property file and json file.
    Change module API to make module as an operator.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
GitHub user tushargosavi reopened a pull request:

    https://github.com/apache/incubator-apex-core/pull/313

    APEXCORE-107 Support for adding modules in the DAG through property and json file

    Support for adding modules in the DAG property file and json file.
    Also changed module API to make module as an operator.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tushargosavi/incubator-apex-core APEXCORE-107

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-apex-core/pull/313.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #313
    
----

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61536617
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java ---
    @@ -2187,6 +2190,27 @@ public void populateDAG(LogicalPlan dag)
     
       }
     
    +  private DagNode addNode(LogicalPlan dag, String name, DagNode nd)
    --- End diff --
    
    "nd" ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61630974
  
    --- Diff: api/src/main/java/com/datatorrent/api/DagNode.java ---
    @@ -0,0 +1,24 @@
    +/**
    + * 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 com.datatorrent.api;
    +
    +/** Marker interface for a node in logical DAG */
    +public interface DagNode
    --- End diff --
    
    Renaming DagNode to Vertex. If I put Vertex in Operator then I can not extend Operator from it, compiler throws cyclic inheritance error.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61794711
  
    --- Diff: api/src/main/java/com/datatorrent/api/Component.java ---
    @@ -116,4 +116,13 @@ public boolean equals(Object obj)
     
       }
     
    +  /**
    +   * Marker interface for the Node in the DAG. Any object which can be added as a Node in the DAG
    +   * needs to implement this interface.
    +   */
    +  interface Vertex
    --- End diff --
    
    moved the Vertex interface to DAG.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi closed the pull request at:

    https://github.com/apache/incubator-apex-core/pull/313


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61536670
  
    --- Diff: engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java ---
    @@ -184,7 +190,7 @@ public void setPortMemory(int portMemory)
         }
       }
     
    -  static class Level2ModuleA implements Module
    +  public static class Level2ModuleA extends BaseOperator implements Module
    --- End diff --
    
    Why these changes?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by PramodSSImmaneni <gi...@git.apache.org>.
Github user PramodSSImmaneni commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-216393279
  
    I also support changing the name to GenericOperator


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61717221
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java ---
    @@ -239,7 +239,7 @@ public void sendMetrics(Collection<String> metricNames)
       {
         @SuppressWarnings("FieldNameHidesFieldInSuperclass")
         private static final long serialVersionUID = 1L;
    -    private OperatorMeta operatorMeta;
    +    private DagNodeMeta operatorMeta;
    --- End diff --
    
    The port can belong to Operator or Module. hence used a common meta object in the portmeta. This common meta object contains only
    - ports 
    - name
    - object
    
    The rest of Operator specific fields are in OperatorMeta and Module specific fields are in ModuleMeta.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61630938
  
    --- Diff: engine/src/test/java/com/datatorrent/stram/plan/logical/module/TestModuleExpansion.java ---
    @@ -184,7 +190,7 @@ public void setPortMemory(int portMemory)
         }
       }
     
    -  static class Level2ModuleA implements Module
    +  public static class Level2ModuleA extends BaseOperator implements Module
    --- End diff --
    
    These changes are required for test cases to pass. without public newInstance invocation fails in addOperator.  There are similar test operator which are defined as public for example TestGeneratorInputOperator
    
    I am removing extends BaseOperator from modules, they were leftovers from previous implementation where module was extending operator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61153475
  
    --- Diff: api/src/main/java/com/datatorrent/api/Module.java ---
    @@ -36,7 +36,7 @@
      * @since 3.3.0
      */
     @InterfaceStability.Evolving
    -public interface Module
    +public interface Module extends Operator
    --- End diff --
    
    The difference design could be having a common interface, which is shared by all type of Nodes in the DAG. Have updated the pull request with this approach.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by PramodSSImmaneni <gi...@git.apache.org>.
Github user PramodSSImmaneni commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-214563277
  
    In the current form, where module gets materialized in the logical plan and does not have a runtime aspect to itself directly, it would be difficult to justify/attribute the lifecycle methods to the module. Maybe module and operator can inherit from a common element which could be the building block of DAG. Thats a bigger change and requires more discussions with wider group.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-apex-core/pull/313


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by davidyan74 <gi...@git.apache.org>.
Github user davidyan74 commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-216389226
  
    I think in the field of streaming technology, all of these different nodes in the DAG share the same concept of an operator. So, instead of introducing a new term "vertex" to the platform, how about renaming this interface to "GenericOperator"?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by PramodSSImmaneni <gi...@git.apache.org>.
Github user PramodSSImmaneni commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61790619
  
    --- Diff: api/src/main/java/com/datatorrent/api/Component.java ---
    @@ -116,4 +116,13 @@ public boolean equals(Object obj)
     
       }
     
    +  /**
    +   * Marker interface for the Node in the DAG. Any object which can be added as a Node in the DAG
    +   * needs to implement this interface.
    +   */
    +  interface Vertex
    --- End diff --
    
    How about moving it into DAG as this is specific to DAG. Component is more general and is implemented by artifacts that are not part of DAG such as security components.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-216571779
  
    changed the name of Vertex to GenericOperator


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61630963
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java ---
    @@ -2187,6 +2190,27 @@ public void populateDAG(LogicalPlan dag)
     
       }
     
    +  private DagNode addNode(LogicalPlan dag, String name, DagNode nd)
    --- End diff --
    
    changed it to vertex.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61672989
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java ---
    @@ -239,7 +239,7 @@ public void sendMetrics(Collection<String> metricNames)
       {
         @SuppressWarnings("FieldNameHidesFieldInSuperclass")
         private static final long serialVersionUID = 1L;
    -    private OperatorMeta operatorMeta;
    +    private DagNodeMeta operatorMeta;
    --- End diff --
    
    keep OperatorMeta


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61778110
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java ---
    @@ -2187,6 +2190,27 @@ public void populateDAG(LogicalPlan dag)
     
       }
     
    +  private Vertex addNode(LogicalPlan dag, String name, Vertex vertex)
    --- End diff --
    
    addOperator


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61794671
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java ---
    @@ -2187,6 +2190,27 @@ public void populateDAG(LogicalPlan dag)
     
       }
     
    +  private Vertex addNode(LogicalPlan dag, String name, Vertex vertex)
    --- End diff --
    
    Changed it to addOperator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by davidyan74 <gi...@git.apache.org>.
Github user davidyan74 commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61787475
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlanConfiguration.java ---
    @@ -2187,6 +2190,27 @@ public void populateDAG(LogicalPlan dag)
     
       }
     
    +  private Vertex addNode(LogicalPlan dag, String name, Vertex vertex)
    --- End diff --
    
    addVertex?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by sandeshh <gi...@git.apache.org>.
Github user sandeshh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61617386
  
    --- Diff: api/src/main/java/com/datatorrent/api/DagNode.java ---
    @@ -0,0 +1,24 @@
    +/**
    + * 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 com.datatorrent.api;
    +
    +/** Marker interface for a node in logical DAG */
    +public interface DagNode
    --- End diff --
    
    dag is no more a dag. Vertex +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61673044
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java ---
    @@ -778,32 +799,162 @@ public void resetStreamPersistanceOnSinkRemoval(InputPortMeta sinkBeingRemoved)
         }
       }
     
    +  public class DagNodeMeta implements Serializable
    +  {
    +    protected final LinkedHashMap<InputPortMeta, StreamMeta> inputStreams = new LinkedHashMap<>();
    +    protected final LinkedHashMap<OutputPortMeta, StreamMeta> outputStreams = new LinkedHashMap<>();
    +    @NotNull
    +    protected final String name;
    +    protected transient Vertex node;
    --- End diff --
    
    node -> operator


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61536347
  
    --- Diff: api/src/main/java/com/datatorrent/api/DagNode.java ---
    @@ -0,0 +1,24 @@
    +/**
    + * 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 com.datatorrent.api;
    +
    +/** Marker interface for a node in logical DAG */
    +public interface DagNode
    --- End diff --
    
    Since ports are defined under Operator, this should probably live there also.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61716868
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java ---
    @@ -778,32 +799,162 @@ public void resetStreamPersistanceOnSinkRemoval(InputPortMeta sinkBeingRemoved)
         }
       }
     
    +  public class DagNodeMeta implements Serializable
    +  {
    +    protected final LinkedHashMap<InputPortMeta, StreamMeta> inputStreams = new LinkedHashMap<>();
    +    protected final LinkedHashMap<OutputPortMeta, StreamMeta> outputStreams = new LinkedHashMap<>();
    +    @NotNull
    +    protected final String name;
    +    protected transient Vertex node;
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by PramodSSImmaneni <gi...@git.apache.org>.
Github user PramodSSImmaneni commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61617019
  
    --- Diff: api/src/main/java/com/datatorrent/api/DagNode.java ---
    @@ -0,0 +1,24 @@
    +/**
    + * 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 com.datatorrent.api;
    +
    +/** Marker interface for a node in logical DAG */
    +public interface DagNode
    --- End diff --
    
    I like vertex. How about stream extending edge :).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61673039
  
    --- Diff: engine/src/main/java/com/datatorrent/stram/plan/logical/LogicalPlan.java ---
    @@ -778,32 +799,162 @@ public void resetStreamPersistanceOnSinkRemoval(InputPortMeta sinkBeingRemoved)
         }
       }
     
    +  public class DagNodeMeta implements Serializable
    +  {
    +    protected final LinkedHashMap<InputPortMeta, StreamMeta> inputStreams = new LinkedHashMap<>();
    +    protected final LinkedHashMap<OutputPortMeta, StreamMeta> outputStreams = new LinkedHashMap<>();
    +    @NotNull
    +    protected final String name;
    +    protected transient Vertex node;
    +
    +    /*
    +     * Used for  OIO validation,
    +     *  value null => node not visited yet
    +     *  other value => represents the root oio node for this node
    +     */
    +    protected transient Integer oioRoot = null;
    +    public DagNodeMeta(String name, Vertex node)
    +    {
    +      this.name = name;
    +      this.node = node;
    +    }
    +
    +    public String getName()
    +    {
    +      return name;
    +    }
    +
    +    public Vertex getNode()
    +    {
    +      return node;
    +    }
    +
    +    private class PortMapping implements Operators.OperatorDescriptor
    +    {
    +      private final Map<Operator.InputPort<?>, InputPortMeta> inPortMap = new HashMap<>();
    +      private final Map<Operator.OutputPort<?>, OutputPortMeta> outPortMap = new HashMap<>();
    +      private final Map<String, Object> portNameMap = new HashMap<>();
    +
    +      @Override
    +      public void addInputPort(InputPort<?> portObject, Field field, InputPortFieldAnnotation portAnnotation, AppData.QueryPort adqAnnotation)
    +      {
    +        if (!DagNodeMeta.this.inputStreams.isEmpty()) {
    +          for (Map.Entry<LogicalPlan.InputPortMeta, LogicalPlan.StreamMeta> e : DagNodeMeta.this.inputStreams.entrySet()) {
    +            LogicalPlan.InputPortMeta pm = e.getKey();
    +            if (pm.operatorMeta == DagNodeMeta.this && pm.fieldName.equals(field.getName())) {
    +              //LOG.debug("Found existing port meta for: " + field);
    +              inPortMap.put(portObject, pm);
    +              markInputPortIfHidden(pm.getPortName(), pm, field.getDeclaringClass());
    +              return;
    +            }
    +          }
    +        }
    +        InputPortMeta metaPort = new InputPortMeta();
    +        metaPort.operatorMeta = DagNodeMeta.this;
    +        metaPort.fieldName = field.getName();
    +        metaPort.portAnnotation = portAnnotation;
    +        metaPort.adqAnnotation = adqAnnotation;
    +        inPortMap.put(portObject, metaPort);
    +        markInputPortIfHidden(metaPort.getPortName(), metaPort, field.getDeclaringClass());
    +      }
    +
    +      @Override
    +      public void addOutputPort(OutputPort<?> portObject, Field field, OutputPortFieldAnnotation portAnnotation, AppData.ResultPort adrAnnotation)
    +      {
    +        if (!DagNodeMeta.this.outputStreams.isEmpty()) {
    +          for (Map.Entry<LogicalPlan.OutputPortMeta, LogicalPlan.StreamMeta> e : DagNodeMeta.this.outputStreams.entrySet()) {
    +            LogicalPlan.OutputPortMeta pm = e.getKey();
    +            if (pm.operatorMeta == DagNodeMeta.this && pm.fieldName.equals(field.getName())) {
    +              //LOG.debug("Found existing port meta for: " + field);
    +              outPortMap.put(portObject, pm);
    +              markOutputPortIfHidden(pm.getPortName(), pm, field.getDeclaringClass());
    +              return;
    +            }
    +          }
    +        }
    +        OutputPortMeta metaPort = new OutputPortMeta();
    +        metaPort.operatorMeta = DagNodeMeta.this;
    +        metaPort.fieldName = field.getName();
    +        metaPort.portAnnotation = portAnnotation;
    +        metaPort.adrAnnotation = adrAnnotation;
    +        outPortMap.put(portObject, metaPort);
    +        markOutputPortIfHidden(metaPort.getPortName(), metaPort, field.getDeclaringClass());
    +      }
    +
    +      private void markOutputPortIfHidden(String portName, OutputPortMeta portMeta, Class<?> declaringClass)
    +      {
    +        if (!portNameMap.containsKey(portName)) {
    +          portNameMap.put(portName, portMeta);
    +        } else {
    +          // make the port optional
    +          portMeta.classDeclaringHiddenPort = declaringClass;
    +        }
    +
    +      }
    +
    +      private void markInputPortIfHidden(String portName, InputPortMeta portMeta, Class<?> declaringClass)
    +      {
    +        if (!portNameMap.containsKey(portName)) {
    +          portNameMap.put(portName, portMeta);
    +        } else {
    +          // make the port optional
    +          portMeta.classDeclaringHiddenPort = declaringClass;
    +        }
    +      }
    +    }
    +
    +    /**
    +     * Ports objects are transient, we keep a lazy initialized mapping
    +     */
    +    private transient PortMapping portMapping = null;
    +
    +    protected PortMapping getPortMapping()
    +    {
    +      if (this.portMapping == null) {
    +        this.portMapping = new PortMapping();
    +        Operators.describe(this.getNode(), portMapping);
    +      }
    +      return portMapping;
    +    }
    +
    +    public OutputPortMeta getMeta(Operator.OutputPort<?> port)
    +    {
    +      return getPortMapping().outPortMap.get(port);
    +    }
    +
    +    public InputPortMeta getMeta(Operator.InputPort<?> port)
    +    {
    +      return getPortMapping().inPortMap.get(port);
    +    }
    +
    +    public Map<InputPortMeta, StreamMeta> getInputStreams()
    +    {
    +      return this.inputStreams;
    +    }
    +
    +    public Map<OutputPortMeta, StreamMeta> getOutputStreams()
    +    {
    +      return this.outputStreams;
    +    }
    +
    +  }
    +
       /**
        * Operator meta object.
        */
    -  public final class OperatorMeta implements DAG.OperatorMeta, Serializable
    +  public class OperatorMeta extends DagNodeMeta implements DAG.OperatorMeta, Serializable
    --- End diff --
    
    Not clear why we need this distinction and why some of the fields are present here and not in "DagNodeMeta".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-217016767
  
    Merged GenericOperatorMeta with OperatorMeta.  ModuleMeta extends from OpeartorMeta. We can merge ModuleMeta and keep a single OperatorMeta but that will involve some changes in more number of files. Let me know if merging ModuleMeta with OperatorMeta if way to go forward.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-215984811
  
    The common interface that we introduce for a vertex in the logical DAG, it could probably need some broader review and discussion. As for the naming, other candidates I have: LogicalDagNode, LogicalOperator.
    
    I prefer to keep the user facing term "operator" stable and consistent. The new interface we discuss is more of an internal detail to process operators in a uniform manner. As such, it would be nice to not even see it as top level interface. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-216165307
  
    Putting Vertex inside Component , to avoid it being as top level interface. As all node component of the graph inherits from Vertex.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61006802
  
    --- Diff: api/src/main/java/com/datatorrent/api/Module.java ---
    @@ -36,7 +36,7 @@
      * @since 3.3.0
      */
     @InterfaceStability.Evolving
    -public interface Module
    +public interface Module extends Operator
    --- End diff --
    
    Operator contains execution life cycle methods, which are not appropriate for "Module". Please consider a different design.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tushargosavi <gi...@git.apache.org>.
Github user tushargosavi commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-214263317
  
    @PramodSSImmaneni @sandeepdeshmukh can you review this pull request?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#issuecomment-214561638
  
    Looks like this already came up during previous discussion and there wasn't any conclusion. Need to go back to that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-apex-core pull request: APEXCORE-107 Support for adding ...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-core/pull/313#discussion_r61536916
  
    --- Diff: api/src/main/java/com/datatorrent/api/DagNode.java ---
    @@ -0,0 +1,24 @@
    +/**
    + * 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 com.datatorrent.api;
    +
    +/** Marker interface for a node in logical DAG */
    +public interface DagNode
    --- End diff --
    
    How about naming it Vertex? Other suggestions? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---