You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by paul-rogers <gi...@git.apache.org> on 2017/04/09 03:52:26 UTC

[GitHub] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

GitHub user paul-rogers opened a pull request:

    https://github.com/apache/drill/pull/811

    DRILL-5423: Refactor ScanBatch to allow unit testing record readers

    Refactors ScanBatch to allow unit testing of record reader
    implementations, especially the \u201cwriter\u201d classes.
    
    See JIRA for details.

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

    $ git pull https://github.com/paul-rogers/drill DRILL-5423

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

    https://github.com/apache/drill/pull/811.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 #811
    
----
commit d8b25c287492f821dce6da6aeb2124be7678e351
Author: Paul Rogers <pr...@maprtech.com>
Date:   2017-04-09T03:52:04Z

    DRILL-5423: Refactor ScanBatch to allow unit testing record readers
    
    Refactors ScanBatch to allow unit testing of record reader
    implementations, especially the \u201cwriter\u201d classes.
    
    See JIRA for details.

----


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114465904
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java ---
    @@ -143,20 +104,18 @@ public void close() {
         }
         logger.debug("Closing context for {}", popConfig != null ? popConfig.getClass().getName() : null);
     
    -    manager.close();
    -
    -    if (allocator != null) {
    -      allocator.close();
    -    }
    -
    -    if (fs != null) {
    -      try {
    -        fs.close();
    -      } catch (IOException e) {
    -        throw new DrillRuntimeException(e);
    +    closed = true;
    --- End diff --
    
    To see this better, view the full source. You'll see these lines:
    ```
      public void close() {
        if (closed) {
          logger.debug("Attempted to close Operator context for {}, but context is already closed", popConfig != null ? popConfig.getClass().getName() : null);
          return;
        }
        logger.debug("Closing context for {}", popConfig != null ? popConfig.getClass().getName() : null);
        closed = true;
    ```
    
    Moving the line up makes it clearer that the only purpose of this flag is to enforce close-once semantics. (The check is original code.)
    
    Before, this function did not call super.close(), which is necessary to close the allocator (which used to be here.)
    
    All closes are wrapped in exceptions. If super.close() fails, we still close the file system.


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114411352
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -323,11 +318,31 @@ public TypedFieldId getValueVectorId(SchemaPath path) {
         return container.getValueAccessorById(clazz, ids);
       }
     
    -  private class Mutator implements OutputMutator {
    +  public static class Mutator implements OutputMutator {
    --- End diff --
    
    Maybe add comments about `Mutator` since the class is `public static`?


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114399420
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorUtilities.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.drill.exec.ops;
    +
    +import java.util.Iterator;
    +
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +
    +public class OperatorUtilities {
    +
    +  private OperatorUtilities() { }
    +
    +  public static int getChildCount(PhysicalOperator popConfig) {
    +    Iterator<PhysicalOperator> iter = popConfig.iterator();
    +    int i = 0;
    --- End diff --
    
    Refactor i -> count


---
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] drill issue #811: DRILL-5423: Refactor ScanBatch to allow unit testing recor...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/811
  
    Rebased on latest master.


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114399171
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorExecContext.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.drill.exec.ops;
    +
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.testing.ExecutionControls;
    +
    +import io.netty.buffer.DrillBuf;
    +
    +/**
    + * Narrowed version of the {@link OpeartorContext} used to create an
    --- End diff --
    
    Typo -> OpeartorContext


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114398903
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java ---
    @@ -143,20 +104,18 @@ public void close() {
         }
         logger.debug("Closing context for {}", popConfig != null ? popConfig.getClass().getName() : null);
     
    -    manager.close();
    -
    -    if (allocator != null) {
    -      allocator.close();
    -    }
    -
    -    if (fs != null) {
    -      try {
    -        fs.close();
    -      } catch (IOException e) {
    -        throw new DrillRuntimeException(e);
    +    closed = true;
    --- End diff --
    
    What is the rationale for moving `closed = true` earlier. What would happen if we have an exception while closing?


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114466127
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorUtilities.java ---
    @@ -0,0 +1,41 @@
    +/*
    + * 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.drill.exec.ops;
    +
    +import java.util.Iterator;
    +
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +
    +public class OperatorUtilities {
    +
    +  private OperatorUtilities() { }
    +
    +  public static int getChildCount(PhysicalOperator popConfig) {
    +    Iterator<PhysicalOperator> iter = popConfig.iterator();
    +    int i = 0;
    --- End diff --
    
    Fixed. This code was actually just moved from its old home in OperatorContext, but went ahead and did the rename as it seems like a good idea.


---
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] drill issue #811: DRILL-5423: Refactor ScanBatch to allow unit testing recor...

Posted by gparai <gi...@git.apache.org>.
Github user gparai commented on the issue:

    https://github.com/apache/drill/pull/811
  
    Thanks for making the changes Paul. 
    
    LGTM +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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114465900
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java ---
    @@ -0,0 +1,94 @@
    +/*
    + * 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.drill.exec.ops;
    +
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.testing.ExecutionControls;
    +
    +import io.netty.buffer.DrillBuf;
    +
    +/**
    + * Implementation of {@link OperatorExecContext} that provides services
    + * needed by most run-time operators. Excludes services that need the
    + * entire Drillbit. Allows easy testing of operator code that uses this
    + * interface.
    + */
    +
    +public class AbstractOperatorExecContext implements OperatorExecContext {
    +
    +  protected final BufferAllocator allocator;
    +  protected final ExecutionControls executionControls;
    +  protected final PhysicalOperator popConfig;
    +  protected final BufferManager manager;
    +  protected OperatorStatReceiver statsWriter;
    +
    +  public AbstractOperatorExecContext(BufferAllocator allocator, PhysicalOperator popConfig,
    +                                     ExecutionControls executionControls,
    +                                     OperatorStatReceiver stats) {
    +    this.allocator = allocator;
    +    this.popConfig = popConfig;
    +    this.manager = new BufferManagerImpl(allocator);
    +    statsWriter = stats;
    +
    +    this.executionControls = executionControls;
    +  }
    +
    +  @Override
    +  public DrillBuf replace(DrillBuf old, int newSize) {
    +    return manager.replace(old, newSize);
    +  }
    +
    +  @Override
    +  public DrillBuf getManagedBuffer() {
    +    return manager.getManagedBuffer();
    +  }
    +
    +  @Override
    +  public DrillBuf getManagedBuffer(int size) {
    +    return manager.getManagedBuffer(size);
    +  }
    +
    +  @Override
    +  public ExecutionControls getExecutionControls() {
    +    return executionControls;
    +  }
    +
    +  @Override
    +  public BufferAllocator getAllocator() {
    +    if (allocator == null) {
    +      throw new UnsupportedOperationException("Operator context does not have an allocator");
    +    }
    +    return allocator;
    +  }
    +
    +  public void close() {
    +    try {
    +      manager.close();
    +    } finally {
    +      if (allocator != null) {
    +        allocator.close();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public OperatorStatReceiver getStatsWriter() {
    +    return statsWriter;
    --- End diff --
    
    The stats writer is a required part of the operator exec context. Said another way, this is the context that operators use for low-level (non-network) services. The stats writer must exist -- either in production code (normal use) or in a test fixture (for unit tests.)
    
    I can make one change, make the member variable final to show that it must be set to one and only one value at construction time.


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114466324
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java ---
    @@ -323,11 +318,31 @@ public TypedFieldId getValueVectorId(SchemaPath path) {
         return container.getValueAccessorById(clazz, ids);
       }
     
    -  private class Mutator implements OutputMutator {
    +  public static class Mutator implements OutputMutator {
    --- End diff --
    
    Done. Also added a `@VisibleForTesting` annotation as well.


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114465941
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorExecContext.java ---
    @@ -0,0 +1,46 @@
    +/*
    + * 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.drill.exec.ops;
    +
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.testing.ExecutionControls;
    +
    +import io.netty.buffer.DrillBuf;
    +
    +/**
    + * Narrowed version of the {@link OpeartorContext} used to create an
    --- End diff --
    
    Fixed.


---
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] drill pull request #811: DRILL-5423: Refactor ScanBatch to allow unit testin...

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

    https://github.com/apache/drill/pull/811#discussion_r114403519
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/ops/AbstractOperatorExecContext.java ---
    @@ -0,0 +1,94 @@
    +/*
    + * 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.drill.exec.ops;
    +
    +import org.apache.drill.exec.memory.BufferAllocator;
    +import org.apache.drill.exec.physical.base.PhysicalOperator;
    +import org.apache.drill.exec.testing.ExecutionControls;
    +
    +import io.netty.buffer.DrillBuf;
    +
    +/**
    + * Implementation of {@link OperatorExecContext} that provides services
    + * needed by most run-time operators. Excludes services that need the
    + * entire Drillbit. Allows easy testing of operator code that uses this
    + * interface.
    + */
    +
    +public class AbstractOperatorExecContext implements OperatorExecContext {
    +
    +  protected final BufferAllocator allocator;
    +  protected final ExecutionControls executionControls;
    +  protected final PhysicalOperator popConfig;
    +  protected final BufferManager manager;
    +  protected OperatorStatReceiver statsWriter;
    +
    +  public AbstractOperatorExecContext(BufferAllocator allocator, PhysicalOperator popConfig,
    +                                     ExecutionControls executionControls,
    +                                     OperatorStatReceiver stats) {
    +    this.allocator = allocator;
    +    this.popConfig = popConfig;
    +    this.manager = new BufferManagerImpl(allocator);
    +    statsWriter = stats;
    +
    +    this.executionControls = executionControls;
    +  }
    +
    +  @Override
    +  public DrillBuf replace(DrillBuf old, int newSize) {
    +    return manager.replace(old, newSize);
    +  }
    +
    +  @Override
    +  public DrillBuf getManagedBuffer() {
    +    return manager.getManagedBuffer();
    +  }
    +
    +  @Override
    +  public DrillBuf getManagedBuffer(int size) {
    +    return manager.getManagedBuffer(size);
    +  }
    +
    +  @Override
    +  public ExecutionControls getExecutionControls() {
    +    return executionControls;
    +  }
    +
    +  @Override
    +  public BufferAllocator getAllocator() {
    +    if (allocator == null) {
    +      throw new UnsupportedOperationException("Operator context does not have an allocator");
    +    }
    +    return allocator;
    +  }
    +
    +  public void close() {
    +    try {
    +      manager.close();
    +    } finally {
    +      if (allocator != null) {
    +        allocator.close();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public OperatorStatReceiver getStatsWriter() {
    +    return statsWriter;
    --- End diff --
    
    This maybe called prior to initializing `statsWriter = stats` in `OperatorContextImpl` constructor. Should we add an assert? 
    OR 
    Should we even pass `OperatorStatReceiver stats` in the `AbstractOperatorExecContext` constructor - maybe only use a getter/setter for `statsWriter`?


---
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.
---