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

ASF GitHub Bot commented on APEXCORE-448:
-----------------------------------------

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

    https://github.com/apache/apex-core/pull/364#discussion_r73433050
  
    --- Diff: 
engine/src/main/java/com/datatorrent/stram/engine/OperatorContext.java ---
    @@ -84,12 +87,13 @@ public void setIdleTimeout(long idleTimeout)
        * @param attributes the value of attributes
        * @param parentContext
        */
    -  public OperatorContext(int id, AttributeMap attributes, Context 
parentContext)
    +  public OperatorContext(int id, String operatorName, AttributeMap 
attributes, Context parentContext)
       {
         super(attributes, parentContext);
         this.lastProcessedWindowId = Stateless.WINDOW_ID;
         this.id = id;
         this.stateless = super.getValue(OperatorContext.STATELESS);
    +    this.operatorName = Preconditions.checkNotNull(operatorName, "operator 
name");
    --- End diff --
    
    @vrozov I am running this app with a ```null``` operator name and 
NullPointerException is thrown:
    ```
      @ApplicationAnnotation(name = "TestApp")
      class SplitterApp implements StreamingApplication
      {
        MockReceiver receiver;
    
        @Override
        public void populateDAG(DAG dag, Configuration configuration)
        {
          dag.setAttribute(DAG.APPLICATION_PATH, baseTestMeta.dataDirectory);
          MockFileInput fileInput = dag.addOperator("Input", new 
MockFileInput());
          fileInput.filePaths = baseTestMeta.filePaths;
    
          FileSplitterBase splitter = dag.addOperator(null, new 
FileSplitterBase());
          splitter.setFile(baseTestMeta.dataDirectory);
    
          receiver = dag.addOperator("Receiver", new MockReceiver());
    
          dag.addStream("files", fileInput.files, splitter.input);
          dag.addStream("file-metadata", splitter.filesMetadataOutput, 
receiver.fileMetadata);
        }
      }
    java.lang.NullPointerException
        at 
com.datatorrent.stram.plan.logical.LogicalPlan$OperatorMeta.equals(LogicalPlan.java:1187)
        at java.util.ArrayList.remove(ArrayList.java:528)
        at 
com.datatorrent.stram.plan.logical.LogicalPlan$StreamMeta.addSink(LogicalPlan.java:531)
        at 
com.datatorrent.stram.plan.logical.LogicalPlan.addStream(LogicalPlan.java:1391)
        at 
com.datatorrent.stram.plan.logical.LogicalPlan.addStream(LogicalPlan.java:1442)
        at 
com.datatorrent.stram.plan.logical.LogicalPlan.addStream(LogicalPlan.java:119)
        at 
com.datatorrent.lib.io.fs.FileSplitterBaseTest$SplitterApp.populateDAG(FileSplitterBaseTest.java:231)
        at 
com.datatorrent.stram.plan.logical.LogicalPlanConfiguration.prepareDAG(LogicalPlanConfiguration.java:2226)
        at com.datatorrent.stram.LocalModeImpl.prepareDAG(LocalModeImpl.java:67)
        at 
com.datatorrent.lib.io.fs.FileSplitterBaseTest.testSplitterInApp(FileSplitterBaseTest.java:205)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47)
        at 
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
        at 
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44)
        at 
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
        at org.junit.rules.TestWatcher$1.evaluate(TestWatcher.java:55)


> Make operator name available in OperatorContext
> -----------------------------------------------
>
>                 Key: APEXCORE-448
>                 URL: https://issues.apache.org/jira/browse/APEXCORE-448
>             Project: Apache Apex Core
>          Issue Type: Improvement
>            Reporter: Chandni Singh
>            Assignee: Chandni Singh
>
> Need name of the logical operator in the OperatorContext which can be used by 
> WindowDataManager to create a unique path per logical operator .



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

Reply via email to