[jira] [Commented] (DRILL-4091) Support more functions in gis contrib module

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-4091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16233593#comment-16233593
 ] 

ASF GitHub Bot commented on DRILL-4091:
---

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

https://github.com/apache/drill/pull/258#discussion_r148172786
  
--- Diff: 
contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnionAggregate.java
 ---
@@ -0,0 +1,114 @@
+/**
+ * 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.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.expr.holders.UInt1Holder;
+
+import com.esri.core.geometry.SpatialReference;
+
+import io.netty.buffer.DrillBuf;
+
+@FunctionTemplate(name = "st_unionaggregate", scope = 
FunctionTemplate.FunctionScope.POINT_AGGREGATE)
+public class STUnionAggregate implements DrillAggFunc {
+  @Param NullableVarBinaryHolder in;
+  @Workspace ObjectHolder value;
--- End diff --

I believe the ObjectHolder is deprecated.  However, it would be worth 
having a discussion about "undeprecating" it as without it, it is really 
difficult if not impossible to write aggregate functions. 


> Support more functions in gis contrib module
> 
>
> Key: DRILL-4091
> URL: https://issues.apache.org/jira/browse/DRILL-4091
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Reporter: Karol Potocki
>Priority: Major
>
> Support for commonly used gis functions in gis contrib module: relate, 
> contains, crosses, intersects, touches, difference, disjoint, buffer, union 
> etc.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-4091) Support more functions in gis contrib module

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-4091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16233591#comment-16233591
 ] 

ASF GitHub Bot commented on DRILL-4091:
---

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

https://github.com/apache/drill/pull/258#discussion_r148172580
  
--- Diff: 
contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STUnionAggregate.java
 ---
@@ -0,0 +1,114 @@
+/**
+ * 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.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillAggFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.IntHolder;
+import org.apache.drill.exec.expr.holders.NullableVarBinaryHolder;
+import org.apache.drill.exec.expr.holders.ObjectHolder;
+import org.apache.drill.exec.expr.holders.UInt1Holder;
+
+import com.esri.core.geometry.SpatialReference;
+
+import io.netty.buffer.DrillBuf;
+
+@FunctionTemplate(name = "st_unionaggregate", scope = 
FunctionTemplate.FunctionScope.POINT_AGGREGATE)
--- End diff --

Do you want to include NULL handling here?


> Support more functions in gis contrib module
> 
>
> Key: DRILL-4091
> URL: https://issues.apache.org/jira/browse/DRILL-4091
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Reporter: Karol Potocki
>Priority: Major
>
> Support for commonly used gis functions in gis contrib module: relate, 
> contains, crosses, intersects, touches, difference, disjoint, buffer, union 
> etc.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-4091) Support more functions in gis contrib module

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-4091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16233590#comment-16233590
 ] 

ASF GitHub Bot commented on DRILL-4091:
---

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

https://github.com/apache/drill/pull/258#discussion_r148117858
  
--- Diff: 
contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STContains.java
 ---
@@ -0,0 +1,62 @@
+/**
+ * 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.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
--- End diff --

Please add a comment about what the function does.  This can be cut/pasted 
from the PostGIS docs if need be. 
IE
`/*Returns true if and only if no points of B lie in the exterior of A, and 
at least one point of the interior of B lies in the interior of A.*/`




> Support more functions in gis contrib module
> 
>
> Key: DRILL-4091
> URL: https://issues.apache.org/jira/browse/DRILL-4091
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Reporter: Karol Potocki
>Priority: Major
>
> Support for commonly used gis functions in gis contrib module: relate, 
> contains, crosses, intersects, touches, difference, disjoint, buffer, union 
> etc.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-4091) Support more functions in gis contrib module

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-4091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16233592#comment-16233592
 ] 

ASF GitHub Bot commented on DRILL-4091:
---

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

https://github.com/apache/drill/pull/258#discussion_r148114646
  
--- Diff: 
contrib/gis/src/main/java/com/esri/core/geometry/VertexGeomAccessor.java ---
@@ -0,0 +1,25 @@
+/**
--- End diff --

Please put the Apache license in a regular java comment instead of a 
Javadoc.  Please update all files accordingly. 


> Support more functions in gis contrib module
> 
>
> Key: DRILL-4091
> URL: https://issues.apache.org/jira/browse/DRILL-4091
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Reporter: Karol Potocki
>Priority: Major
>
> Support for commonly used gis functions in gis contrib module: relate, 
> contains, crosses, intersects, touches, difference, disjoint, buffer, union 
> etc.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-4091) Support more functions in gis contrib module

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-4091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16233594#comment-16233594
 ] 

ASF GitHub Bot commented on DRILL-4091:
---

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

https://github.com/apache/drill/pull/258#discussion_r148115797
  
--- Diff: 
contrib/gis/src/main/java/org/apache/drill/exec/expr/fn/impl/gis/STBuffer.java 
---
@@ -0,0 +1,65 @@
+/**
+ * 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.expr.fn.impl.gis;
+
+import javax.inject.Inject;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.Float8Holder;
+import org.apache.drill.exec.expr.holders.VarBinaryHolder;
+
+import io.netty.buffer.DrillBuf;
+
+@FunctionTemplate(name = "st_buffer", scope = 
FunctionTemplate.FunctionScope.SIMPLE,
+  nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+public class STBuffer implements DrillSimpleFunc {
+  @Param
+  VarBinaryHolder geom1Param;
+
+  @Param(constant = true)
--- End diff --

Are you setting a constant that is used in multiple function iterations?  
If so, I've not seen it done this way before.  The way I've done this is by 
using a Java primitive in the `@Workspace` and setting the value in the 
`setup()` function.  

Is there some advantage to doing it this way?


> Support more functions in gis contrib module
> 
>
> Key: DRILL-4091
> URL: https://issues.apache.org/jira/browse/DRILL-4091
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Reporter: Karol Potocki
>Priority: Major
>
> Support for commonly used gis functions in gis contrib module: relate, 
> contains, crosses, intersects, touches, difference, disjoint, buffer, union 
> etc.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5842) Refactor and simplify the fragment, operator contexts for testing

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5842?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16233528#comment-16233528
 ] 

ASF GitHub Bot commented on DRILL-5842:
---

Github user sohami commented on the issue:

https://github.com/apache/drill/pull/978
  
Looking into the PR and based on description it says below:

_A result of these changes is that OperatorContext is now a one-stop-shop 
for services needed by operators. It now provides:_

- Access to the fragment context: getFragmentContext()
- Access to the physical operator definition (AKA “physical operator”): 
getOperatorDefn().

_Because of these changes, we need no longer pass around the triple of 
fragment context, operator definition and operator context — something needed 
by the “managed” sort and upcoming PRs._

Based on my understanding it looks like existing interface and 
implementation `OperExecContextImpl` do provide access to `FragmentContext` and 
`PopConfig` (operator Defn). The instance of this context is created inside 
`ExternalSortBatch` which has access to FragmentContext and PopConfig already. 
Also `SortImpl` only takes in `OperExecContext`. I am little confused how the 
change avoided the need to pass around the triplet ?


> Refactor and simplify the fragment, operator contexts for testing
> -
>
> Key: DRILL-5842
> URL: https://issues.apache.org/jira/browse/DRILL-5842
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.12.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
>Priority: Major
> Fix For: 1.12.0
>
>
> Drill's execution engine has a "fragment context" that provides state for a 
> fragment as a whole, and an "operator context" which provides state for a 
> single operator. Historically, these have both been concrete classes that 
> make generous references to the Drillbit context, and hence need a full Drill 
> server in order to operate.
> Drill has historically made extensive use of system-level testing: build the 
> entire server and fire queries at it to test each component. Over time, we 
> are augmenting that approach with unit tests: the ability to test each 
> operator (or parts of an operator) in isolation.
> Since each operator requires access to both the operator and fragment 
> context, the fact that the contexts depend on the overall server creates a 
> large barrier to unit testing. An earlier checkin started down the path of 
> defining the contexts as interfaces that can have different run-time and 
> test-time implementations to enable testing.
> This ticket asks to refactor those interfaces: simplifying the operator 
> context and introducing an interface for the fragment context. New code will 
> use these new interfaces, while older code continues to use the concrete 
> implementations. Over time, as operators are enhanced, they can be modified 
> to allow unit-level testing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5842) Refactor and simplify the fragment, operator contexts for testing

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5842?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227814#comment-16227814
 ] 

ASF GitHub Bot commented on DRILL-5842:
---

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

https://github.com/apache/drill/pull/978#discussion_r148146597
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java 
---
@@ -17,33 +17,23 @@
  */
 package org.apache.drill.exec.ops;
 
-import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
 
-import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.exception.OutOfMemoryException;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
 import org.apache.drill.exec.work.WorkManager;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 
-import com.google.common.base.Preconditions;
 import com.google.common.util.concurrent.ListenableFuture;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 
-class OperatorContextImpl extends AbstractOperatorExecContext implements 
OperatorContext, AutoCloseable {
+class OperatorContextImpl extends BaseOperatorContext implements 
OperatorContext, AutoCloseable {
   static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(OperatorContextImpl.class);
--- End diff --

Why we need to have `OperatorContextImpl implements OperatorContext` when 
`BaseOperatorContex`t already implements that interface ?


> Refactor and simplify the fragment, operator contexts for testing
> -
>
> Key: DRILL-5842
> URL: https://issues.apache.org/jira/browse/DRILL-5842
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.12.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.12.0
>
>
> Drill's execution engine has a "fragment context" that provides state for a 
> fragment as a whole, and an "operator context" which provides state for a 
> single operator. Historically, these have both been concrete classes that 
> make generous references to the Drillbit context, and hence need a full Drill 
> server in order to operate.
> Drill has historically made extensive use of system-level testing: build the 
> entire server and fire queries at it to test each component. Over time, we 
> are augmenting that approach with unit tests: the ability to test each 
> operator (or parts of an operator) in isolation.
> Since each operator requires access to both the operator and fragment 
> context, the fact that the contexts depend on the overall server creates a 
> large barrier to unit testing. An earlier checkin started down the path of 
> defining the contexts as interfaces that can have different run-time and 
> test-time implementations to enable testing.
> This ticket asks to refactor those interfaces: simplifying the operator 
> context and introducing an interface for the fragment context. New code will 
> use these new interfaces, while older code continues to use the concrete 
> implementations. Over time, as operators are enhanced, they can be modified 
> to allow unit-level testing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5842) Refactor and simplify the fragment, operator contexts for testing

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5842?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227815#comment-16227815
 ] 

ASF GitHub Bot commented on DRILL-5842:
---

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

https://github.com/apache/drill/pull/978#discussion_r148095543
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java 
---
@@ -0,0 +1,201 @@
+/*
+ * 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.io.IOException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Preconditions;
+
+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. This class provides services common to the test-time
+ * version of the operator context and the full production-time context
+ * that includes network services.
+ */
+
+public abstract class BaseOperatorContext implements OperatorContext {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BaseOperatorContext.class);
+
+  protected final FragmentContextInterface context;
+  protected final BufferAllocator allocator;
+  protected final PhysicalOperator popConfig;
+  protected final BufferManager manager;
+  protected OperatorStatReceiver statsWriter;
+  private DrillFileSystem fs;
+  private ControlsInjector injector;
+
+  public BaseOperatorContext(FragmentContextInterface context, 
BufferAllocator allocator,
+   PhysicalOperator popConfig,
+   OperatorStatReceiver stats) {
+this.context = context;
+this.allocator = allocator;
+this.popConfig = popConfig;
+this.manager = new BufferManagerImpl(allocator);
+statsWriter = stats;
+  }
+
+  @Override
+  public FragmentContextInterface getFragmentContext() {
+return context;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public  T getOperatorDefn() {
+return (T) popConfig;
+  }
+
+  public String getName() {
+return popConfig.getClass().getName();
+  }
+
+  @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 context.getExecutionControls();
+  }
+
+  @Override
+  public BufferAllocator getAllocator() {
+if (allocator == null) {
+  throw new UnsupportedOperationException("Operator context does not 
have an allocator");
+}
+return allocator;
+  }
+
+  // Allow an operator to use the thread pool
+  @Override
+  public ExecutorService getExecutor() {
+return context.getDrillbitContext().getExecutor();
+  }
+
+  @Override
+  public ExecutorService getScanExecutor() {
+return context.getDrillbitContext().getScanExecutor();
+  }
+
+  @Override
+  public ExecutorService getScanDecodeExecutor() {
+return context.getDrillbitContext().getScanDecodeExecutor();
+  }
+
+  @Override
+  public 

[jira] [Commented] (DRILL-5842) Refactor and simplify the fragment, operator contexts for testing

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5842?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227812#comment-16227812
 ] 

ASF GitHub Bot commented on DRILL-5842:
---

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

https://github.com/apache/drill/pull/978#discussion_r148150382
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ops/BaseOperatorContext.java 
---
@@ -0,0 +1,201 @@
+/*
+ * 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.io.IOException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.testing.ControlsInjector;
+import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.hadoop.conf.Configuration;
+
+import com.google.common.base.Preconditions;
+
+import io.netty.buffer.DrillBuf;
+
+/**
+ * Implementation of {@link OperatorExecContext} that provides services
--- End diff --

`OperatorContext` instead of _OperatorExecContext_


> Refactor and simplify the fragment, operator contexts for testing
> -
>
> Key: DRILL-5842
> URL: https://issues.apache.org/jira/browse/DRILL-5842
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.12.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.12.0
>
>
> Drill's execution engine has a "fragment context" that provides state for a 
> fragment as a whole, and an "operator context" which provides state for a 
> single operator. Historically, these have both been concrete classes that 
> make generous references to the Drillbit context, and hence need a full Drill 
> server in order to operate.
> Drill has historically made extensive use of system-level testing: build the 
> entire server and fire queries at it to test each component. Over time, we 
> are augmenting that approach with unit tests: the ability to test each 
> operator (or parts of an operator) in isolation.
> Since each operator requires access to both the operator and fragment 
> context, the fact that the contexts depend on the overall server creates a 
> large barrier to unit testing. An earlier checkin started down the path of 
> defining the contexts as interfaces that can have different run-time and 
> test-time implementations to enable testing.
> This ticket asks to refactor those interfaces: simplifying the operator 
> context and introducing an interface for the fragment context. New code will 
> use these new interfaces, while older code continues to use the concrete 
> implementations. Over time, as operators are enhanced, they can be modified 
> to allow unit-level testing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5842) Refactor and simplify the fragment, operator contexts for testing

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5842?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227813#comment-16227813
 ] 

ASF GitHub Bot commented on DRILL-5842:
---

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

https://github.com/apache/drill/pull/978#discussion_r148153093
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ops/OperatorContextImpl.java 
---
@@ -98,37 +70,25 @@ public boolean isClosed() {
   @Override
   public void close() {
 if (closed) {
-  logger.debug("Attempted to close Operator context for {}, but 
context is already closed", popConfig != null ? popConfig.getClass().getName() 
: null);
+  logger.debug("Attempted to close Operator context for {}, but 
context is already closed", popConfig != null ? getName() : null);
   return;
 }
-logger.debug("Closing context for {}", popConfig != null ? 
popConfig.getClass().getName() : null);
+logger.debug("Closing context for {}", popConfig != null ? getName() : 
null);
 
 closed = true;
--- End diff --

`closed = true` should happen after the call to `super.close()` since base 
class close can throw exception.


> Refactor and simplify the fragment, operator contexts for testing
> -
>
> Key: DRILL-5842
> URL: https://issues.apache.org/jira/browse/DRILL-5842
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.12.0
>Reporter: Paul Rogers
>Assignee: Paul Rogers
> Fix For: 1.12.0
>
>
> Drill's execution engine has a "fragment context" that provides state for a 
> fragment as a whole, and an "operator context" which provides state for a 
> single operator. Historically, these have both been concrete classes that 
> make generous references to the Drillbit context, and hence need a full Drill 
> server in order to operate.
> Drill has historically made extensive use of system-level testing: build the 
> entire server and fire queries at it to test each component. Over time, we 
> are augmenting that approach with unit tests: the ability to test each 
> operator (or parts of an operator) in isolation.
> Since each operator requires access to both the operator and fragment 
> context, the fact that the contexts depend on the overall server creates a 
> large barrier to unit testing. An earlier checkin started down the path of 
> defining the contexts as interfaces that can have different run-time and 
> test-time implementations to enable testing.
> This ticket asks to refactor those interfaces: simplifying the operator 
> context and introducing an interface for the fragment context. New code will 
> use these new interfaces, while older code continues to use the concrete 
> implementations. Over time, as operators are enhanced, they can be modified 
> to allow unit-level testing.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5783) Make code generation in the TopN operator more modular and test it

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5783?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227798#comment-16227798
 ] 

ASF GitHub Bot commented on DRILL-5783:
---

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

https://github.com/apache/drill/pull/984#discussion_r148157170
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/test/BatchUtils.java ---
@@ -0,0 +1,280 @@
+/*
+ * 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.test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.ValueVector;
+import org.junit.Assert;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+public class BatchUtils {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchUtils.class);
+
+  public static Map 
containerToObjects(VectorContainer vectorContainer) {
+Map rows = Maps.newHashMap();
+int numCols = vectorContainer.getNumberOfColumns();
+int recordCount = vectorContainer.getRecordCount();
+
+for (int columnIndex = 0; columnIndex < numCols; columnIndex++) {
+  String columnName = 
vectorContainer.getSchema().getColumn(columnIndex).getName();
+  List data = Lists.newArrayList();
+
+  ValueVector.Accessor valueVectorAccessor = vectorContainer
+.getValueVector(columnIndex)
+.getValueVector()
+.getAccessor();
+
+  for (int recordIndex = 0; recordIndex < recordCount; recordIndex++) {
+data.add(valueVectorAccessor.getObject(recordIndex));
+  }
+
+  rows.put(columnName, data);
+}
+
+return rows;
+  }
+
+  public static Map 
hyperBatchAndSelectorToObjects(VectorContainer vectorContainer, 
SelectionVector4 selectionVector4) {
+Map rows = Maps.newHashMap();
+int numCols = vectorContainer.getNumberOfColumns();
+int numIndices = selectionVector4.getCount();
+
+for (int columnIndex = 0; columnIndex < numCols; columnIndex++) {
+  String columnName = 
vectorContainer.getSchema().getColumn(columnIndex).getName();
+  List data = Lists.newArrayList();
+
+  VectorWrapper vectorWrapper = 
vectorContainer.getValueVector(columnIndex);
+
+  for (int indexIndex = 0; indexIndex < numIndices; indexIndex++) {
+int sv4Index = selectionVector4.get(indexIndex);
+int batchIndex = SelectionVector4.getBatchIndex(sv4Index);
+int recordIndex = SelectionVector4.getRecordIndex(sv4Index);
+
+ValueVector valueVector = 
vectorWrapper.getValueVectors()[batchIndex];
+Object columnValue = 
valueVector.getAccessor().getObject(recordIndex);
+data.add(columnValue);
+  }
+
+  rows.put(columnName, data);
+}
+
+return rows;
+  }
+
+  public static String toString(Map table) {
+if (table.isEmpty()) {
+  return "[ empty table ]";
+}
+
+List columnNames = Lists.newArrayList(table.keySet());
+Collections.sort(columnNames);
+int numRecords = table.get(columnNames.get(0)).size();
+
+StringBuilder sb = new StringBuilder();
+
+{
+  sb.append("[ ");
+  String separator = "";
+
+  for (String columnName : columnNames) {
+sb.append(separator);
+separator = ", ";
+sb.append(columnName);
+  }
+
+  sb.append(" ]\n");
+}
+
+for (int 

[jira] [Commented] (DRILL-5783) Make code generation in the TopN operator more modular and test it

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5783?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227797#comment-16227797
 ] 

ASF GitHub Bot commented on DRILL-5783:
---

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

https://github.com/apache/drill/pull/984#discussion_r148157140
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/test/BatchUtils.java ---
@@ -0,0 +1,280 @@
+/*
+ * 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.test;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.drill.exec.record.VectorContainer;
+import org.apache.drill.exec.record.VectorWrapper;
+import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.apache.drill.exec.vector.ValueVector;
+import org.junit.Assert;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+public class BatchUtils {
+  static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(BatchUtils.class);
+
+  public static Map 
containerToObjects(VectorContainer vectorContainer) {
+Map rows = Maps.newHashMap();
+int numCols = vectorContainer.getNumberOfColumns();
+int recordCount = vectorContainer.getRecordCount();
+
+for (int columnIndex = 0; columnIndex < numCols; columnIndex++) {
+  String columnName = 
vectorContainer.getSchema().getColumn(columnIndex).getName();
+  List data = Lists.newArrayList();
+
+  ValueVector.Accessor valueVectorAccessor = vectorContainer
+.getValueVector(columnIndex)
+.getValueVector()
+.getAccessor();
+
+  for (int recordIndex = 0; recordIndex < recordCount; recordIndex++) {
+data.add(valueVectorAccessor.getObject(recordIndex));
+  }
+
+  rows.put(columnName, data);
+}
+
+return rows;
+  }
+
+  public static Map 
hyperBatchAndSelectorToObjects(VectorContainer vectorContainer, 
SelectionVector4 selectionVector4) {
--- End diff --

Done


> Make code generation in the TopN operator more modular and test it
> --
>
> Key: DRILL-5783
> URL: https://issues.apache.org/jira/browse/DRILL-5783
> Project: Apache Drill
>  Issue Type: Improvement
>Reporter: Timothy Farkas
>Assignee: Timothy Farkas
>




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5783) Make code generation in the TopN operator more modular and test it

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5783?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227708#comment-16227708
 ] 

ASF GitHub Bot commented on DRILL-5783:
---

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

https://github.com/apache/drill/pull/984#discussion_r148144235
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/test/BaseDirTestWatcher.java ---
@@ -0,0 +1,184 @@
+/*
+ * 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.test;
+
+import com.google.common.base.Charsets;
+import org.apache.commons.io.FileUtils;
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.util.TestUtilities;
+import org.junit.runner.Description;
+
+import java.io.File;
+import java.io.IOException;
+
+public class BaseDirTestWatcher extends DirTestWatcher {
+  public enum DirType {
+ROOT,
+TEST_TMP;
+  }
+
+  private File tmpDir;
+  private File storeDir;
+  private File dfsTestTmpParentDir;
+  private File dfsTestTmpDir;
+  private File rootDir;
+
+  public BaseDirTestWatcher() {
+super();
+  }
+
+  @Override
+  protected void starting(Description description) {
+super.starting(description);
+
+rootDir = makeSubDir("root");
+tmpDir = new File(rootDir, "tmp");
+storeDir = new File(rootDir, "store");
+dfsTestTmpParentDir = new File(rootDir, "dfsTestTmp");
+
+tmpDir.mkdirs();
+storeDir.mkdirs();
+dfsTestTmpParentDir.mkdirs();
+  }
+
+  public File getTmpDir() {
+return tmpDir;
+  }
+
+  public File getStoreDir() {
+return storeDir;
+  }
+
+  public File getDfsTestTmpParentDir() {
+return dfsTestTmpParentDir;
+  }
+
+  public File getDfsTestTmpDir() {
+return dfsTestTmpDir;
+  }
+
+  public String getDfsTestTmpDirPath() {
+return dfsTestTmpDir.getAbsolutePath().replaceAll("/\\./", "/");
+  }
+
+  public File getRootDir() {
+return rootDir;
+  }
+
+  public String getRootDirPath() {
+return rootDir.getAbsolutePath().replaceAll("/\\./", "/");
+  }
+
+  public void newDfsTestTmpDir() {
+dfsTestTmpDir = 
TestUtilities.createTempDir(BaseTestQuery.dirTestWatcher.getDfsTestTmpParentDir());
+  }
+
+  private File getDir(DirType type) {
+switch (type) {
+  case ROOT:
+return rootDir;
+  case TEST_TMP:
+return dfsTestTmpDir;
+  default:
+throw new IllegalArgumentException(String.format("Unsupported type 
%s", type));
+}
+  }
+
+  public File makeRootSubDir(String relPath) {
+return makeSubDir(relPath, DirType.ROOT);
+  }
+
+  public File makeTestTmpSubDir(String relPath) {
+return makeSubDir(relPath, DirType.TEST_TMP);
+  }
+
+  private File makeSubDir(String relPath, DirType type) {
+File subDir = new File(getDir(type), relPath);
+subDir.mkdirs();
+return subDir;
+  }
+
+  /**
+   * This preserves the relative path of the directory in root
+   * @param relPath
+   * @return
+   */
+  public File copyResourceToRoot(String relPath) {
+return copyTo(relPath, relPath, TestTools.DataType.RESOURCE, 
DirType.ROOT);
+  }
+
+  public File copyFileToRoot(String relPath) {
+return copyTo(relPath, relPath, TestTools.DataType.PROJECT, 
DirType.ROOT);
+  }
+
+  public File copyResourceToRoot(String relPath, String destPath) {
+return copyTo(relPath, destPath, TestTools.DataType.RESOURCE, 
DirType.ROOT);
+  }
+
+  public File copyResourceToTestTmp(String relPath, String destPath) {
+return copyTo(relPath, destPath, TestTools.DataType.RESOURCE, 
DirType.TEST_TMP);
+  }
+
+  private File copyTo(String relPath, String destPath, TestTools.DataType 

[jira] [Commented] (DRILL-5468) TPCH Q18 regressed ~3x due to execution plan changes

2017-10-31 Thread Dechang Gu (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5468?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227570#comment-16227570
 ] 

Dechang Gu commented on DRILL-5468:
---

I tried this setting on SF100, Q18 did not show the 3x regression, performance 
is on par with previous normal runs.

> TPCH Q18 regressed ~3x due to execution plan changes
> 
>
> Key: DRILL-5468
> URL: https://issues.apache.org/jira/browse/DRILL-5468
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Functions - Drill
>Affects Versions: 1.11.0
> Environment: 10+1 node ucs-micro cluster RHEL6.4
>Reporter: Dechang Gu
>Assignee: Jinfeng Ni
> Attachments: Q18_profile_gitid_841ead4, Q18_profile_gitid_adbf363
>
>
> In a regular regression test on Drill master (commit id 841ead4) TPCH Q18 on 
> SF100 parquet dataset took ~81 secs, while the same query on 1.10.0 took only 
> ~27 secs.  The query time on the commit adbf363 which is right before 841ead4 
> is ~32 secs.
> Profiles shows the plans for the query changed quite a bit (profiles will be 
> uploaded) 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5863) Sortable table incorrectly sorts minor fragments and time elements lexically instead of sorting by implicit value

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227565#comment-16227565
 ] 

ASF GitHub Bot commented on DRILL-5863:
---

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

https://github.com/apache/drill/pull/987#discussion_r148126237
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
 ---
@@ -76,12 +77,14 @@ public String getId() {
   public String getContent() {
 TableBuilder builder = new TableBuilder(OPERATOR_COLUMNS, 
OPERATOR_COLUMNS_TOOLTIP, true);
 
+Map attributeMap = new HashMap(); 
//Reusing for different fragments
 for (ImmutablePair, String> ip 
: opsAndHosts) {
   int minor = ip.getLeft().getRight();
   OperatorProfile op = ip.getLeft().getLeft();
 
+  attributeMap.put("data-order", String.valueOf(minor)); //Overwrite 
values from previous fragments
--- End diff --

You are correct that the values must be zero padded. However, the padding 
by default is only up till achieving a width of two digits. The sequencing 
didn't get messed up as long as the number of minor fragments for each major 
fragment was less than 100.
For greater than 100, the sequencing broke the way you described it.
So, until this fix, you'll get the minor fragment ordered as: 
01-01-XX . 01-10-XX, 01-100-XX, 
instead of 
01-01-XX . 01-10-XX, 01-11-XX,  01-10-XX, 01-11-XX,  01-100-XX, 
01-101-XX

What we are doing is injecting the  element with an attribute that 
carries the actual numeric value. The dataTable library will discover and sort 
the elements of a column by the value of this attribute instead of the value 
that the element encapsulates.


> Sortable table incorrectly sorts minor fragments and time elements lexically 
> instead of sorting by implicit value
> -
>
> Key: DRILL-5863
> URL: https://issues.apache.org/jira/browse/DRILL-5863
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Web Server
>Affects Versions: 1.12.0
>Reporter: Kunal Khatua
>Assignee: Kunal Khatua
>Priority: Minor
> Fix For: 1.12.0
>
>
> The fix for this is to use dataTable library's {{data-order}} attribute for 
> the data elements that need to sort by an implicit value.
> ||Old order of Minor Fragment||New order of Minor Fragment||
> |...|...|
> |01-09-01  | 01-09-01|
> |01-10-01  | 01-10-01|
> |01-100-01 | 01-11-01|
> |01-101-01 | 01-12-01|
> |... | ... |
> ||Old order of Duration||New order of Duration|||
> |...|...|
> |1m15s  | 55.03s|
> |55s  | 1m15s|
> |...|...|



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5863) Sortable table incorrectly sorts minor fragments and time elements lexically instead of sorting by implicit value

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227527#comment-16227527
 ] 

ASF GitHub Bot commented on DRILL-5863:
---

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

https://github.com/apache/drill/pull/987#discussion_r148123715
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/OperatorWrapper.java
 ---
@@ -76,12 +77,14 @@ public String getId() {
   public String getContent() {
 TableBuilder builder = new TableBuilder(OPERATOR_COLUMNS, 
OPERATOR_COLUMNS_TOOLTIP, true);
 
+Map attributeMap = new HashMap(); 
//Reusing for different fragments
 for (ImmutablePair, String> ip 
: opsAndHosts) {
   int minor = ip.getLeft().getRight();
   OperatorProfile op = ip.getLeft().getLeft();
 
+  attributeMap.put("data-order", String.valueOf(minor)); //Overwrite 
values from previous fragments
   String path = new 
OperatorPathBuilder().setMajor(major).setMinor(minor).setOperator(op).build();
-  builder.appendCell(path);
+  builder.appendCell(path, attributeMap);
--- End diff --

The appendCell() method has been overloaded with to allow for a map, from 
which any set of attributes can be injected. 

https://github.com/kkhatua/drill/blob/783ca992f3bbb52ae606a8ca10745d6dd2ed0d4f/exec/java-exec/src/main/java/org/apache/drill/exec/server/rest/profile/TableBuilder.java#L95
This allows for any future libraries that might need to inject their own 
set of attributes to be incorporated without the need for any significant 
modification to the TableBuilder.
So, each appendCell is provided its own unique map that is consumed during 
the construction of the `` element


> Sortable table incorrectly sorts minor fragments and time elements lexically 
> instead of sorting by implicit value
> -
>
> Key: DRILL-5863
> URL: https://issues.apache.org/jira/browse/DRILL-5863
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Web Server
>Affects Versions: 1.12.0
>Reporter: Kunal Khatua
>Assignee: Kunal Khatua
>Priority: Minor
> Fix For: 1.12.0
>
>
> The fix for this is to use dataTable library's {{data-order}} attribute for 
> the data elements that need to sort by an implicit value.
> ||Old order of Minor Fragment||New order of Minor Fragment||
> |...|...|
> |01-09-01  | 01-09-01|
> |01-10-01  | 01-10-01|
> |01-100-01 | 01-11-01|
> |01-101-01 | 01-12-01|
> |... | ... |
> ||Old order of Duration||New order of Duration|||
> |...|...|
> |1m15s  | 55.03s|
> |55s  | 1m15s|
> |...|...|



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (DRILL-5918) Allow manual configuration when using Runtime.getRuntime().availableProcessors()

2017-10-31 Thread Elijah Zupancic (JIRA)
Elijah Zupancic created DRILL-5918:
--

 Summary: Allow manual configuration when using 
Runtime.getRuntime().availableProcessors()
 Key: DRILL-5918
 URL: https://issues.apache.org/jira/browse/DRILL-5918
 Project: Apache Drill
  Issue Type: Improvement
Affects Versions: 1.11.0
Reporter: Elijah Zupancic
Priority: Critical


Currently Drill auto-configures the number of threads in various thread pools 
based on the processor count:

https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java#L110
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java#L269
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/AbstractResourceManager.java#L55
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/rm/DefaultResourceManager.java#L99

In a number of situations, this configuration is incorrect. In particular, the 
settings aren't correct when [running in a bare-metal 
container](https://docs.google.com/document/d/1WGPGiJtbJZPZBMSD9YHAAz-O4AlAET9qOnS_GeriZtE/edit?usp=sharing)
 because you don't necessarily have access to all of the compute shares or 
memory of the underlying host.

Ideally, the number of CPUs to use could be specified as a system parameter to 
Drill. This would allow users to configure Drill to their optimal settings.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227349#comment-16227349
 ] 

Vitalii Diravka commented on DRILL-5822:


[~Paul.Rogers] 
Here is the issue of unnecessary sorting of columns for query with the 
following conditions: using wildcard in the query and ORDER BY clause, and when 
this is planned into multiple fragments ("alter session set 
`planner.slice_target`=1;").

The issue is connected to adding canonicalizing the schemas of input batches 
for Merging Receiver in DRILL-847. But this approach is outdated since for now 
in the process of loading batches in the RecordBatchLoader the new batch with 
same columns (SchemaPaths) but other ordering of them is perceived with same 
schema as the previous batch has: 
[All fields from the last batch is a hashMap 
structure|https://github.com/apache/drill/blob/fe79a633a3da8b4f6db50454fde64c30c73233bb/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java#L90]
 and [when new batch appears the columns are just removed from the old one by 
the 
key|https://github.com/apache/drill/blob/fe79a633a3da8b4f6db50454fde64c30c73233bb/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java#L102].
So the schemaChange flag still equals to false. And then [the schema will 
built|https://github.com/apache/drill/blob/fe79a633a3da8b4f6db50454fde64c30c73233bb/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java#L138].
 

Here is only the issue that RecordBatchLoader permutes column order for the 
above case. And it was described in the jira ticket created by you DRILL-5828 
and can be fixed there.
So my changes fix the current issue but not fully cover the requirements from 
your comment. Will It be reasonably if that changes will be done in context of 
DRILL-5828?

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (DRILL-5166) Select with options returns NPE

2017-10-31 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5166:

Fix Version/s: (was: Future)

> Select with options returns NPE
> ---
>
> Key: DRILL-5166
> URL: https://issues.apache.org/jira/browse/DRILL-5166
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Fix For: Future
>
>
> When querying two csv files:
> First file (2 records):
> {noformat}
> key_header, value_header
> key_1,value_1
> {noformat}
> Second file (50 records):
> {noformat}
> key_header, value_header
> key_1,value_1
> ...
> key_49,value_49
> {noformat}
> Select with options returns NPE:
> {noformat}
> select * from table(dfs.root.`/home/arina/files/ver/*.csv`(type => 
> 'text',extractHeader => true, fieldDelimiter => ',')) limit 10;
> {noformat}
> Querying without options works file:
> {noformat}
> select  * from dfs.root.`/home/arina/files/ver/*.csv` limit 10;
> {noformat}
> Error:
> {noformat}
> Caused by: org.apache.drill.common.exceptions.UserRemoteException: SYSTEM 
> ERROR: NullPointerException
> Fragment 1:0
> [Error Id: b789f5f8-f090-4097-b7ff-9f4efd3d01e8 on localhost:31013]
>   (com.fasterxml.jackson.databind.JsonMappingException) Instantiation of 
> [simple type, class org.apache.drill.exec.store.dfs.easy.EasySubScan] value 
> failed (java.lang.NullPointerException): null
>  at [Source: {
>   "pop" : "single-sender",
>   "@id" : 0,
>   "receiver-major-fragment" : 0,
>   "receiver-minor-fragment" : 0,
>   "child" : {
> "pop" : "selection-vector-remover",
> "@id" : 1,
> "child" : {
>   "pop" : "limit",
>   "@id" : 2,
>   "child" : {
> "pop" : "fs-sub-scan",
> "@id" : 3,
> "userName" : "arina",
> "files" : [ {
>   "start" : 0,
>   "length" : 11777804,
>   "path" : "file:/home/arina/files/ver/key_value_50.csv"
> } ],
> "storage" : {
>   "type" : "file",
>   "enabled" : true,
>   "connection" : "file:///",
>   "config" : null,
>   "workspaces" : {
> "root" : {
>   "location" : "/",
>   "writable" : false,
>   "defaultInputFormat" : null
> },
> "tmp" : {
>   "location" : "/tmp",
>   "writable" : false,
>   "defaultInputFormat" : null
> }
>   },
>   "formats" : {
> "psv" : {
>   "type" : "text",
>   "extensions" : [ "tbl" ],
>   "delimiter" : "|"
> },
> "csv" : {
>   "type" : "text",
>   "extensions" : [ "csv" ],
>   "delimiter" : ","
> },
> "tsv" : {
>   "type" : "text",
>   "extensions" : [ "tsv" ],
>   "delimiter" : "\t"
> },
> "httpd" : {
>   "type" : "httpd",
>   "logFormat" : "%h %t \"%r\" %>s %b \"%{Referer}i\"",
>   "timestampFormat" : null
> },
> "parquet" : {
>   "type" : "parquet"
> },
> "json" : {
>   "type" : "json",
>   "extensions" : [ "json" ]
> },
> "avro" : {
>   "type" : "avro"
> },
> "sequencefile" : {
>   "type" : "sequencefile",
>   "extensions" : [ "seq" ]
> },
> "csvh" : {
>   "type" : "text",
>   "extensions" : [ "csvh" ],
>   "extractHeader" : true,
>   "delimiter" : ","
> }
>   }
> },
> "format" : {
>   "type" : "named",
>   "name" : "text"
> },
> "columns" : [ "`*`" ],
> "selectionRoot" : "file:/home/arina/files/ver",
> "initialAllocation" : 100,
> "maxAllocation" : 100,
> "cost" : 0.0
>   },
>   "first" : 0,
>   "last" : 10,
>   "initialAllocation" : 100,
>   "maxAllocation" : 100,
>   "cost" : 10.0
> },
> "initialAllocation" : 100,
> "maxAllocation" : 100,
> "cost" : 10.0
>   },
>   "destination" : "Cglsb2NhbGhvc3QQpfIBGKbyASCn8gEyDzEuMTAuMC1TTkFQU0hPVA==",
>   "initialAllocation" : 100,
>   "maxAllocation" : 100,
>   "cost" : 10.0
> }; line: 90, column: 7] (through reference chain: 
> org.apache.drill.exec.physical.config.SingleSender["child"]->org.apache.drill.exec.physical.config.SelectionVectorRemover["child"]->org.apache.drill.exec.physical.config.Limit["child"])
> com.fasterxml.jackson.databind.JsonMappingException.from():223
>  

[jira] [Updated] (DRILL-5166) Select with options returns NPE

2017-10-31 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5166?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5166:

Fix Version/s: Future

> Select with options returns NPE
> ---
>
> Key: DRILL-5166
> URL: https://issues.apache.org/jira/browse/DRILL-5166
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.9.0
>Reporter: Arina Ielchiieva
>Assignee: Arina Ielchiieva
> Fix For: Future
>
>
> When querying two csv files:
> First file (2 records):
> {noformat}
> key_header, value_header
> key_1,value_1
> {noformat}
> Second file (50 records):
> {noformat}
> key_header, value_header
> key_1,value_1
> ...
> key_49,value_49
> {noformat}
> Select with options returns NPE:
> {noformat}
> select * from table(dfs.root.`/home/arina/files/ver/*.csv`(type => 
> 'text',extractHeader => true, fieldDelimiter => ',')) limit 10;
> {noformat}
> Querying without options works file:
> {noformat}
> select  * from dfs.root.`/home/arina/files/ver/*.csv` limit 10;
> {noformat}
> Error:
> {noformat}
> Caused by: org.apache.drill.common.exceptions.UserRemoteException: SYSTEM 
> ERROR: NullPointerException
> Fragment 1:0
> [Error Id: b789f5f8-f090-4097-b7ff-9f4efd3d01e8 on localhost:31013]
>   (com.fasterxml.jackson.databind.JsonMappingException) Instantiation of 
> [simple type, class org.apache.drill.exec.store.dfs.easy.EasySubScan] value 
> failed (java.lang.NullPointerException): null
>  at [Source: {
>   "pop" : "single-sender",
>   "@id" : 0,
>   "receiver-major-fragment" : 0,
>   "receiver-minor-fragment" : 0,
>   "child" : {
> "pop" : "selection-vector-remover",
> "@id" : 1,
> "child" : {
>   "pop" : "limit",
>   "@id" : 2,
>   "child" : {
> "pop" : "fs-sub-scan",
> "@id" : 3,
> "userName" : "arina",
> "files" : [ {
>   "start" : 0,
>   "length" : 11777804,
>   "path" : "file:/home/arina/files/ver/key_value_50.csv"
> } ],
> "storage" : {
>   "type" : "file",
>   "enabled" : true,
>   "connection" : "file:///",
>   "config" : null,
>   "workspaces" : {
> "root" : {
>   "location" : "/",
>   "writable" : false,
>   "defaultInputFormat" : null
> },
> "tmp" : {
>   "location" : "/tmp",
>   "writable" : false,
>   "defaultInputFormat" : null
> }
>   },
>   "formats" : {
> "psv" : {
>   "type" : "text",
>   "extensions" : [ "tbl" ],
>   "delimiter" : "|"
> },
> "csv" : {
>   "type" : "text",
>   "extensions" : [ "csv" ],
>   "delimiter" : ","
> },
> "tsv" : {
>   "type" : "text",
>   "extensions" : [ "tsv" ],
>   "delimiter" : "\t"
> },
> "httpd" : {
>   "type" : "httpd",
>   "logFormat" : "%h %t \"%r\" %>s %b \"%{Referer}i\"",
>   "timestampFormat" : null
> },
> "parquet" : {
>   "type" : "parquet"
> },
> "json" : {
>   "type" : "json",
>   "extensions" : [ "json" ]
> },
> "avro" : {
>   "type" : "avro"
> },
> "sequencefile" : {
>   "type" : "sequencefile",
>   "extensions" : [ "seq" ]
> },
> "csvh" : {
>   "type" : "text",
>   "extensions" : [ "csvh" ],
>   "extractHeader" : true,
>   "delimiter" : ","
> }
>   }
> },
> "format" : {
>   "type" : "named",
>   "name" : "text"
> },
> "columns" : [ "`*`" ],
> "selectionRoot" : "file:/home/arina/files/ver",
> "initialAllocation" : 100,
> "maxAllocation" : 100,
> "cost" : 0.0
>   },
>   "first" : 0,
>   "last" : 10,
>   "initialAllocation" : 100,
>   "maxAllocation" : 100,
>   "cost" : 10.0
> },
> "initialAllocation" : 100,
> "maxAllocation" : 100,
> "cost" : 10.0
>   },
>   "destination" : "Cglsb2NhbGhvc3QQpfIBGKbyASCn8gEyDzEuMTAuMC1TTkFQU0hPVA==",
>   "initialAllocation" : 100,
>   "maxAllocation" : 100,
>   "cost" : 10.0
> }; line: 90, column: 7] (through reference chain: 
> org.apache.drill.exec.physical.config.SingleSender["child"]->org.apache.drill.exec.physical.config.SelectionVectorRemover["child"]->org.apache.drill.exec.physical.config.Limit["child"])
> com.fasterxml.jackson.databind.JsonMappingException.from():223
> 
> 

[jira] [Assigned] (DRILL-5875) user mixed up problem in apache drill

2017-10-31 Thread Pritesh Maker (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5875?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Pritesh Maker reassigned DRILL-5875:


Assignee: Arina Ielchiieva

> user mixed up problem in apache drill
> -
>
> Key: DRILL-5875
> URL: https://issues.apache.org/jira/browse/DRILL-5875
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.8.0, 1.9.0, 1.10.0, 1.11.0
>Reporter: flyfantasy
>Assignee: Arina Ielchiieva
>
> Hi guys.
> we have used drill for quite a long time.   We used apache-drill-1.8.0 at the 
> beginning and recently we upgraded to apache-drill-1.11.0. Drill is great and 
> now drill already have 40+ user in our company. It accelerate olap queries 
> quite a lot. But as the number of  drill user is getting bigger and bigger, a 
> problem we called user mix-up is getting more and more serious. 
> Let me explain the problem. We are using drill with user impersonation. 
> Different user have different privileges. As we have many drill user, so is 
> it quite common two or more people are using drill at the same time. A user 
> we called u1 posted a query to table t1 located in hdfs which he has 
> privilege through drill and may get an error which tells him that he has no 
> privilege to the table as he was u2 ( another user). " And u2 may get a 
> similar error with his query. The only thing u1 can do in this situation is 
> to exit drill and reconnect to drill through a new session.
> This problem occurs quite frequently.  It occur in apache-drill-1.8.0 and 
> also in apache-drill-1.11.0. User get confused and maybe frustrated while 
> data security is under threaten. 
> PS: we are running drill on a 8 nodes cluster which will connect to a 100 
> nodes hadoop cluster. Hadoop version is 2.6.3. Drill version is 1.11.0. Below 
> is drill-override.conf
> drill.exec: {
>   cluster-id: "olap-drill",
>   zk.connect: 
> "zk01:2181/olap_drill,zk02:2181/olap_drill,zk03:2181/olap_drill",
>security.user.auth: {enabled: true,
> packages += "org.apache.drill.exec.rpc.user.security",
> impl: "pam"
> pam_profiles: ["login", "sudo"]
>},
>impersonation: {
>  enabled: true,
>  max_chained_user_hops: 3
>}
> }
> Thanks for your attention. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Paul Rogers (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227074#comment-16227074
 ] 

Paul Rogers edited comment on DRILL-5822 at 10/31/17 4:41 PM:
--

The general rule for the SQL project clause is the following:

* If the list is explicit, {{SELECT b, c, a}} then columns are returned in that 
order, even if the table defines them in the order (a, b, c).
* If the lis is implicit using a wildcard, {{SELECT *}}, then the column order 
is that defined by the table. In our example above, the order would be {{a, b, 
c}}.

Since Drill is distributed and schema-on-read, we run into the issue that two 
tables might have the same columns, but defined in different orders. For 
example:

{noformat}
Table 1: {"a": 10, "b": 20, "c": 30}
Table 2: {"c": 40, "b": 50, "a": 60}
{noformat}

In this case, there is no "correct" order. Instead, Drill must:

1. Recognize that the above scenario can occur.
2. Define each merging operator to follow some reconciliation rule.

Here a "merging" operator is anything that can see batches from two distinct 
scans. That is, almost all operators, but at least the receivers.

A good reconciliation rule is that the first schema wins, and all other batches 
are projected into that first schema. In our example, {{a, b, c}} and {{c, b, 
a}} are both projected into {{a, b, c}}.

The PMC has asked that we not discuss design issues in PR reviews on Github. 
So, can you perhaps please explain here the approach that this PR takes to 
solve the problem? Do we agree on the description above? Or, did this PR take a 
different approach?


was (Author: paul.rogers):
The general rule for the SQL project clause is the following:

* If the list is explicit, `SELECT b, c, a` then columns are returned in that 
order, even if the table defines them in the order (a, b, c).
* If the lis is implicit using a wildcard, `SELECT *`, then the column order is 
that defined by the table. In our example above, the order would be `a, b, c`.

Since Drill is distributed and schema-on-read, we run into the issue that two 
tables might have the same columns, but defined in different orders. For 
example, `{"a": 10, "b": 20, "c": 30}` and `{"c": 40, "b": 50, "c": 60}`. In 
this case, there is no "correct" order. Instead, Drill must:

1. Recognize that the above scenario can occur.
2. Define each merging operator to follow some reconciliation rule.

Here a "merging" operator is anything that can see batches from two distinct 
scans. That is, almost all operators, but at least the receivers.

A good reconciliation rule is that the first schema wins, and all other batches 
are projected into that first schema. In our example, `a, b, c` and `c, b, a` 
are both projected into `a, b, c`.

The PMC has asked that we not discuss design issues in PR reviews. So, can you 
perhaps please explain here the approach that this PR takes to solve the 
problem? Do we agree on the description above? Or, did this PR take a different 
approach?

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary   

[jira] [Commented] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Paul Rogers (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227074#comment-16227074
 ] 

Paul Rogers commented on DRILL-5822:


The general rule for the SQL project clause is the following:

* If the list is explicit, `SELECT b, c, a` then columns are returned in that 
order, even if the table defines them in the order (a, b, c).
* If the lis is implicit using a wildcard, `SELECT *`, then the column order is 
that defined by the table. In our example above, the order would be `a, b, c`.

Since Drill is distributed and schema-on-read, we run into the issue that two 
tables might have the same columns, but defined in different orders. For 
example, `{"a": 10, "b": 20, "c": 30}` and `{"c": 40, "b": 50, "c": 60}`. In 
this case, there is no "correct" order. Instead, Drill must:

1. Recognize that the above scenario can occur.
2. Define each merging operator to follow some reconciliation rule.

Here a "merging" operator is anything that can see batches from two distinct 
scans. That is, almost all operators, but at least the receivers.

A good reconciliation rule is that the first schema wins, and all other batches 
are projected into that first schema. In our example, `a, b, c` and `c, b, a` 
are both projected into `a, b, c`.

The PMC has asked that we not discuss design issues in PR reviews. So, can you 
perhaps please explain here the approach that this PR takes to solve the 
problem? Do we agree on the description above? Or, did this PR take a different 
approach?

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Assigned] (DRILL-5878) TableNotFound exception is being reported for a wrong storage plugin.

2017-10-31 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva reassigned DRILL-5878:
---

Assignee: Hanumath Rao Maduri  (was: Arina Ielchiieva)

> TableNotFound exception is being reported for a wrong storage plugin.
> -
>
> Key: DRILL-5878
> URL: https://issues.apache.org/jira/browse/DRILL-5878
> Project: Apache Drill
>  Issue Type: Bug
>  Components: SQL Parser
>Affects Versions: 1.11.0
>Reporter: Hanumath Rao Maduri
>Assignee: Hanumath Rao Maduri
>Priority: Minor
>  Labels: ready-to-commit
> Fix For: 1.12.0
>
>
> Drill is reporting TableNotFound exception for a wrong storage plugin. 
> Consider the following query where employee.json is queried using cp plugin.
> {code}
> 0: jdbc:drill:zk=local> select * from cp.`employee.json` limit 10;
> +--++-++--+-+---++-++--++---+-+-++
> | employee_id  | full_name  | first_name  | last_name  | position_id  
> | position_title  | store_id  | department_id  | birth_date  |   
> hire_date|  salary  | supervisor_id  |  education_level  | 
> marital_status  | gender  |  management_role   |
> +--++-++--+-+---++-++--++---+-+-++
> | 1| Sheri Nowmer   | Sheri   | Nowmer | 1
> | President   | 0 | 1  | 1961-08-26  | 
> 1994-12-01 00:00:00.0  | 8.0  | 0  | Graduate Degree   | S
>| F   | Senior Management  |
> | 2| Derrick Whelply| Derrick | Whelply| 2
> | VP Country Manager  | 0 | 1  | 1915-07-03  | 
> 1994-12-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | M
>| M   | Senior Management  |
> | 4| Michael Spence | Michael | Spence | 2
> | VP Country Manager  | 0 | 1  | 1969-06-20  | 
> 1998-01-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | S
>| M   | Senior Management  |
> | 5| Maya Gutierrez | Maya| Gutierrez  | 2
> | VP Country Manager  | 0 | 1  | 1951-05-10  | 
> 1998-01-01 00:00:00.0  | 35000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 6| Roberta Damstra| Roberta | Damstra| 3
> | VP Information Systems  | 0 | 2  | 1942-10-08  | 
> 1994-12-01 00:00:00.0  | 25000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 7| Rebecca Kanagaki   | Rebecca | Kanagaki   | 4
> | VP Human Resources  | 0 | 3  | 1949-03-27  | 
> 1994-12-01 00:00:00.0  | 15000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 8| Kim Brunner| Kim | Brunner| 11   
> | Store Manager   | 9 | 11 | 1922-08-10  | 
> 1998-01-01 00:00:00.0  | 1.0  | 5  | Bachelors Degree  | S
>| F   | Store Management   |
> | 9| Brenda Blumberg| Brenda  | Blumberg   | 11   
> | Store Manager   | 21| 11 | 1979-06-23  | 
> 1998-01-01 00:00:00.0  | 17000.0  | 5  | Graduate Degree   | M
>| F   | Store Management   |
> | 10   | Darren Stanz   | Darren  | Stanz  | 5
> | VP Finance  | 0 | 5  | 1949-08-26  | 
> 1994-12-01 00:00:00.0  | 5.0  | 1  | Partial College   | M
>| M   | Senior Management  |
> | 11   | Jonathan Murraiin  | Jonathan| Murraiin   | 11   
> | Store Manager   | 1 | 11 | 1967-06-20  | 
> 1998-01-01 00:00:00.0  | 15000.0  | 5  | Graduate Degree   | S
>| M   | Store Management   |
> +--++-++--+-+---++-++--++---+-+-++
> {code}
> 

[jira] [Assigned] (DRILL-5878) TableNotFound exception is being reported for a wrong storage plugin.

2017-10-31 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva reassigned DRILL-5878:
---

Assignee: Arina Ielchiieva  (was: Hanumath Rao Maduri)

> TableNotFound exception is being reported for a wrong storage plugin.
> -
>
> Key: DRILL-5878
> URL: https://issues.apache.org/jira/browse/DRILL-5878
> Project: Apache Drill
>  Issue Type: Bug
>  Components: SQL Parser
>Affects Versions: 1.11.0
>Reporter: Hanumath Rao Maduri
>Assignee: Arina Ielchiieva
>Priority: Minor
>  Labels: ready-to-commit
> Fix For: 1.12.0
>
>
> Drill is reporting TableNotFound exception for a wrong storage plugin. 
> Consider the following query where employee.json is queried using cp plugin.
> {code}
> 0: jdbc:drill:zk=local> select * from cp.`employee.json` limit 10;
> +--++-++--+-+---++-++--++---+-+-++
> | employee_id  | full_name  | first_name  | last_name  | position_id  
> | position_title  | store_id  | department_id  | birth_date  |   
> hire_date|  salary  | supervisor_id  |  education_level  | 
> marital_status  | gender  |  management_role   |
> +--++-++--+-+---++-++--++---+-+-++
> | 1| Sheri Nowmer   | Sheri   | Nowmer | 1
> | President   | 0 | 1  | 1961-08-26  | 
> 1994-12-01 00:00:00.0  | 8.0  | 0  | Graduate Degree   | S
>| F   | Senior Management  |
> | 2| Derrick Whelply| Derrick | Whelply| 2
> | VP Country Manager  | 0 | 1  | 1915-07-03  | 
> 1994-12-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | M
>| M   | Senior Management  |
> | 4| Michael Spence | Michael | Spence | 2
> | VP Country Manager  | 0 | 1  | 1969-06-20  | 
> 1998-01-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | S
>| M   | Senior Management  |
> | 5| Maya Gutierrez | Maya| Gutierrez  | 2
> | VP Country Manager  | 0 | 1  | 1951-05-10  | 
> 1998-01-01 00:00:00.0  | 35000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 6| Roberta Damstra| Roberta | Damstra| 3
> | VP Information Systems  | 0 | 2  | 1942-10-08  | 
> 1994-12-01 00:00:00.0  | 25000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 7| Rebecca Kanagaki   | Rebecca | Kanagaki   | 4
> | VP Human Resources  | 0 | 3  | 1949-03-27  | 
> 1994-12-01 00:00:00.0  | 15000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 8| Kim Brunner| Kim | Brunner| 11   
> | Store Manager   | 9 | 11 | 1922-08-10  | 
> 1998-01-01 00:00:00.0  | 1.0  | 5  | Bachelors Degree  | S
>| F   | Store Management   |
> | 9| Brenda Blumberg| Brenda  | Blumberg   | 11   
> | Store Manager   | 21| 11 | 1979-06-23  | 
> 1998-01-01 00:00:00.0  | 17000.0  | 5  | Graduate Degree   | M
>| F   | Store Management   |
> | 10   | Darren Stanz   | Darren  | Stanz  | 5
> | VP Finance  | 0 | 5  | 1949-08-26  | 
> 1994-12-01 00:00:00.0  | 5.0  | 1  | Partial College   | M
>| M   | Senior Management  |
> | 11   | Jonathan Murraiin  | Jonathan| Murraiin   | 11   
> | Store Manager   | 1 | 11 | 1967-06-20  | 
> 1998-01-01 00:00:00.0  | 15000.0  | 5  | Graduate Degree   | S
>| M   | Store Management   |
> +--++-++--+-+---++-++--++---+-+-++
> {code}
> 

[jira] [Updated] (DRILL-5878) TableNotFound exception is being reported for a wrong storage plugin.

2017-10-31 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5878:

Labels: ready-to-commit  (was: )

> TableNotFound exception is being reported for a wrong storage plugin.
> -
>
> Key: DRILL-5878
> URL: https://issues.apache.org/jira/browse/DRILL-5878
> Project: Apache Drill
>  Issue Type: Bug
>  Components: SQL Parser
>Affects Versions: 1.11.0
>Reporter: Hanumath Rao Maduri
>Assignee: Arina Ielchiieva
>Priority: Minor
>  Labels: ready-to-commit
> Fix For: 1.12.0
>
>
> Drill is reporting TableNotFound exception for a wrong storage plugin. 
> Consider the following query where employee.json is queried using cp plugin.
> {code}
> 0: jdbc:drill:zk=local> select * from cp.`employee.json` limit 10;
> +--++-++--+-+---++-++--++---+-+-++
> | employee_id  | full_name  | first_name  | last_name  | position_id  
> | position_title  | store_id  | department_id  | birth_date  |   
> hire_date|  salary  | supervisor_id  |  education_level  | 
> marital_status  | gender  |  management_role   |
> +--++-++--+-+---++-++--++---+-+-++
> | 1| Sheri Nowmer   | Sheri   | Nowmer | 1
> | President   | 0 | 1  | 1961-08-26  | 
> 1994-12-01 00:00:00.0  | 8.0  | 0  | Graduate Degree   | S
>| F   | Senior Management  |
> | 2| Derrick Whelply| Derrick | Whelply| 2
> | VP Country Manager  | 0 | 1  | 1915-07-03  | 
> 1994-12-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | M
>| M   | Senior Management  |
> | 4| Michael Spence | Michael | Spence | 2
> | VP Country Manager  | 0 | 1  | 1969-06-20  | 
> 1998-01-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | S
>| M   | Senior Management  |
> | 5| Maya Gutierrez | Maya| Gutierrez  | 2
> | VP Country Manager  | 0 | 1  | 1951-05-10  | 
> 1998-01-01 00:00:00.0  | 35000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 6| Roberta Damstra| Roberta | Damstra| 3
> | VP Information Systems  | 0 | 2  | 1942-10-08  | 
> 1994-12-01 00:00:00.0  | 25000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 7| Rebecca Kanagaki   | Rebecca | Kanagaki   | 4
> | VP Human Resources  | 0 | 3  | 1949-03-27  | 
> 1994-12-01 00:00:00.0  | 15000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 8| Kim Brunner| Kim | Brunner| 11   
> | Store Manager   | 9 | 11 | 1922-08-10  | 
> 1998-01-01 00:00:00.0  | 1.0  | 5  | Bachelors Degree  | S
>| F   | Store Management   |
> | 9| Brenda Blumberg| Brenda  | Blumberg   | 11   
> | Store Manager   | 21| 11 | 1979-06-23  | 
> 1998-01-01 00:00:00.0  | 17000.0  | 5  | Graduate Degree   | M
>| F   | Store Management   |
> | 10   | Darren Stanz   | Darren  | Stanz  | 5
> | VP Finance  | 0 | 5  | 1949-08-26  | 
> 1994-12-01 00:00:00.0  | 5.0  | 1  | Partial College   | M
>| M   | Senior Management  |
> | 11   | Jonathan Murraiin  | Jonathan| Murraiin   | 11   
> | Store Manager   | 1 | 11 | 1967-06-20  | 
> 1998-01-01 00:00:00.0  | 15000.0  | 5  | Graduate Degree   | S
>| M   | Store Management   |
> +--++-++--+-+---++-++--++---+-+-++
> {code}
> However if cp1 is used instead 

[jira] [Updated] (DRILL-5878) TableNotFound exception is being reported for a wrong storage plugin.

2017-10-31 Thread Arina Ielchiieva (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5878?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Arina Ielchiieva updated DRILL-5878:

Reviewer: Arina Ielchiieva

> TableNotFound exception is being reported for a wrong storage plugin.
> -
>
> Key: DRILL-5878
> URL: https://issues.apache.org/jira/browse/DRILL-5878
> Project: Apache Drill
>  Issue Type: Bug
>  Components: SQL Parser
>Affects Versions: 1.11.0
>Reporter: Hanumath Rao Maduri
>Assignee: Hanumath Rao Maduri
>Priority: Minor
>  Labels: ready-to-commit
> Fix For: 1.12.0
>
>
> Drill is reporting TableNotFound exception for a wrong storage plugin. 
> Consider the following query where employee.json is queried using cp plugin.
> {code}
> 0: jdbc:drill:zk=local> select * from cp.`employee.json` limit 10;
> +--++-++--+-+---++-++--++---+-+-++
> | employee_id  | full_name  | first_name  | last_name  | position_id  
> | position_title  | store_id  | department_id  | birth_date  |   
> hire_date|  salary  | supervisor_id  |  education_level  | 
> marital_status  | gender  |  management_role   |
> +--++-++--+-+---++-++--++---+-+-++
> | 1| Sheri Nowmer   | Sheri   | Nowmer | 1
> | President   | 0 | 1  | 1961-08-26  | 
> 1994-12-01 00:00:00.0  | 8.0  | 0  | Graduate Degree   | S
>| F   | Senior Management  |
> | 2| Derrick Whelply| Derrick | Whelply| 2
> | VP Country Manager  | 0 | 1  | 1915-07-03  | 
> 1994-12-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | M
>| M   | Senior Management  |
> | 4| Michael Spence | Michael | Spence | 2
> | VP Country Manager  | 0 | 1  | 1969-06-20  | 
> 1998-01-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | S
>| M   | Senior Management  |
> | 5| Maya Gutierrez | Maya| Gutierrez  | 2
> | VP Country Manager  | 0 | 1  | 1951-05-10  | 
> 1998-01-01 00:00:00.0  | 35000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 6| Roberta Damstra| Roberta | Damstra| 3
> | VP Information Systems  | 0 | 2  | 1942-10-08  | 
> 1994-12-01 00:00:00.0  | 25000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 7| Rebecca Kanagaki   | Rebecca | Kanagaki   | 4
> | VP Human Resources  | 0 | 3  | 1949-03-27  | 
> 1994-12-01 00:00:00.0  | 15000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 8| Kim Brunner| Kim | Brunner| 11   
> | Store Manager   | 9 | 11 | 1922-08-10  | 
> 1998-01-01 00:00:00.0  | 1.0  | 5  | Bachelors Degree  | S
>| F   | Store Management   |
> | 9| Brenda Blumberg| Brenda  | Blumberg   | 11   
> | Store Manager   | 21| 11 | 1979-06-23  | 
> 1998-01-01 00:00:00.0  | 17000.0  | 5  | Graduate Degree   | M
>| F   | Store Management   |
> | 10   | Darren Stanz   | Darren  | Stanz  | 5
> | VP Finance  | 0 | 5  | 1949-08-26  | 
> 1994-12-01 00:00:00.0  | 5.0  | 1  | Partial College   | M
>| M   | Senior Management  |
> | 11   | Jonathan Murraiin  | Jonathan| Murraiin   | 11   
> | Store Manager   | 1 | 11 | 1967-06-20  | 
> 1998-01-01 00:00:00.0  | 15000.0  | 5  | Graduate Degree   | S
>| M   | Store Management   |
> +--++-++--+-+---++-++--++---+-+-++
> {code}
> However if cp1 is used instead of 

[jira] [Commented] (DRILL-5878) TableNotFound exception is being reported for a wrong storage plugin.

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227026#comment-16227026
 ] 

ASF GitHub Bot commented on DRILL-5878:
---

Github user arina-ielchiieva commented on the issue:

https://github.com/apache/drill/pull/996
  
@HanumathRao thanks for making the changes. 
+1, LGTM.


> TableNotFound exception is being reported for a wrong storage plugin.
> -
>
> Key: DRILL-5878
> URL: https://issues.apache.org/jira/browse/DRILL-5878
> Project: Apache Drill
>  Issue Type: Bug
>  Components: SQL Parser
>Affects Versions: 1.11.0
>Reporter: Hanumath Rao Maduri
>Assignee: Hanumath Rao Maduri
>Priority: Minor
> Fix For: 1.12.0
>
>
> Drill is reporting TableNotFound exception for a wrong storage plugin. 
> Consider the following query where employee.json is queried using cp plugin.
> {code}
> 0: jdbc:drill:zk=local> select * from cp.`employee.json` limit 10;
> +--++-++--+-+---++-++--++---+-+-++
> | employee_id  | full_name  | first_name  | last_name  | position_id  
> | position_title  | store_id  | department_id  | birth_date  |   
> hire_date|  salary  | supervisor_id  |  education_level  | 
> marital_status  | gender  |  management_role   |
> +--++-++--+-+---++-++--++---+-+-++
> | 1| Sheri Nowmer   | Sheri   | Nowmer | 1
> | President   | 0 | 1  | 1961-08-26  | 
> 1994-12-01 00:00:00.0  | 8.0  | 0  | Graduate Degree   | S
>| F   | Senior Management  |
> | 2| Derrick Whelply| Derrick | Whelply| 2
> | VP Country Manager  | 0 | 1  | 1915-07-03  | 
> 1994-12-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | M
>| M   | Senior Management  |
> | 4| Michael Spence | Michael | Spence | 2
> | VP Country Manager  | 0 | 1  | 1969-06-20  | 
> 1998-01-01 00:00:00.0  | 4.0  | 1  | Graduate Degree   | S
>| M   | Senior Management  |
> | 5| Maya Gutierrez | Maya| Gutierrez  | 2
> | VP Country Manager  | 0 | 1  | 1951-05-10  | 
> 1998-01-01 00:00:00.0  | 35000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 6| Roberta Damstra| Roberta | Damstra| 3
> | VP Information Systems  | 0 | 2  | 1942-10-08  | 
> 1994-12-01 00:00:00.0  | 25000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 7| Rebecca Kanagaki   | Rebecca | Kanagaki   | 4
> | VP Human Resources  | 0 | 3  | 1949-03-27  | 
> 1994-12-01 00:00:00.0  | 15000.0  | 1  | Bachelors Degree  | M
>| F   | Senior Management  |
> | 8| Kim Brunner| Kim | Brunner| 11   
> | Store Manager   | 9 | 11 | 1922-08-10  | 
> 1998-01-01 00:00:00.0  | 1.0  | 5  | Bachelors Degree  | S
>| F   | Store Management   |
> | 9| Brenda Blumberg| Brenda  | Blumberg   | 11   
> | Store Manager   | 21| 11 | 1979-06-23  | 
> 1998-01-01 00:00:00.0  | 17000.0  | 5  | Graduate Degree   | M
>| F   | Store Management   |
> | 10   | Darren Stanz   | Darren  | Stanz  | 5
> | VP Finance  | 0 | 5  | 1949-08-26  | 
> 1994-12-01 00:00:00.0  | 5.0  | 1  | Partial College   | M
>| M   | Senior Management  |
> | 11   | Jonathan Murraiin  | Jonathan| Murraiin   | 11   
> | Store Manager   | 1 | 11 | 1967-06-20  | 
> 1998-01-01 00:00:00.0  | 15000.0  | 5  | Graduate Degree   | S
>| M   | Store Management   |
> 

[jira] [Commented] (DRILL-5834) Add Networking Functions

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16227020#comment-16227020
 ] 

ASF GitHub Bot commented on DRILL-5834:
---

Github user arina-ielchiieva commented on the issue:

https://github.com/apache/drill/pull/971
  
@cgivre under squash commits @paul-rogers meant that you should end up with 
one commit (now you have 13). You might consider using rebase and then do force 
push in your branch to overwrite this 13 commits. Also please make sure your 
commit message has proper naming: DRILL-XXX: . In your case, 
`DRILL-5834: Add networking functions`.


> Add Networking Functions
> 
>
> Key: DRILL-5834
> URL: https://issues.apache.org/jira/browse/DRILL-5834
> Project: Apache Drill
>  Issue Type: Improvement
>  Components: Functions - Drill
>Affects Versions: 1.11.0
>Reporter: Charles Givre
>Assignee: Charles Givre
>Priority: Minor
>  Labels: doc-impacting
> Fix For: 1.12.0
>
>
> On the heels of the PCAP plugin, this is a collection of functions that would 
> facilitate network analysis using Drill. 
> The functions include:
> inet_aton(): Converts an IPv4 address into an integer.
> inet_ntoa( ): Converts an integer IP into dotted decimal notation
> in_network( , ): Returns true if the IP address is in the given 
> CIDR block
> address_count(  ): Returns the number of IPs in a given CIDR block
> broadcast_address(  ): Returns the broadcast address for a given CIDR 
> block
> netmask( ): Returns the netmask for a given CIDR block.
> low_address(): Returns the first address in a given CIDR block.
> high_address(): Returns the last address in a given CIDR block.
> url_encode(  ): Returns a URL encoded string.
> url_decode(  ): Decodes a URL encoded string.
> is_valid_IP(): Returns true if the IP is a valid IP address
> is_private_ip(): Returns true if the IP is a private IPv4 address
> is_valid_IPv4(): Returns true if the IP is a valid IPv4 address
> is_valid_IPv6(): Returns true if the IP is a valid IPv6 address



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5917) Ban json.org library in Drill

2017-10-31 Thread Arina Ielchiieva (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5917?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226997#comment-16226997
 ] 

Arina Ielchiieva commented on DRILL-5917:
-

FYI
[~priteshm], [~amansinha100], [~vrozov]

> Ban json.org library in Drill
> -
>
> Key: DRILL-5917
> URL: https://issues.apache.org/jira/browse/DRILL-5917
> Project: Apache Drill
>  Issue Type: Task
>Affects Versions: 1.11.0
>Reporter: Arina Ielchiieva
> Fix For: 1.12.0
>
>
> Apache Drill has dependencies on json.org lib indirectly from two libraries:
> com.mapr.hadoop:maprfs:jar:5.2.1-mapr
> com.mapr.fs:mapr-hbase:jar:5.2.1-mapr
> {noformat}
> [INFO] org.apache.drill.contrib:drill-format-mapr:jar:1.12.0-SNAPSHOT
> [INFO] +- com.mapr.hadoop:maprfs:jar:5.2.1-mapr:compile
> [INFO] |  \- org.json:json:jar:20080701:compile
> [INFO] \- com.mapr.fs:mapr-hbase:jar:5.2.1-mapr:compile
> [INFO]\- (org.json:json:jar:20080701:compile - omitted for duplicate)
> {noformat}
> Need to make sure we won't have any dependencies from these libs to json.org 
> lib and ban this lib in main pom.xml file.
> Issue is critical since Apache release won't happen until we make sure 
> json.org lib is not used (https://www.apache.org/legal/resolved.html).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (DRILL-5917) Ban json.org library in Drill

2017-10-31 Thread Arina Ielchiieva (JIRA)
Arina Ielchiieva created DRILL-5917:
---

 Summary: Ban json.org library in Drill
 Key: DRILL-5917
 URL: https://issues.apache.org/jira/browse/DRILL-5917
 Project: Apache Drill
  Issue Type: Task
Affects Versions: 1.11.0
Reporter: Arina Ielchiieva
 Fix For: 1.12.0


Apache Drill has dependencies on json.org lib indirectly from two libraries:
com.mapr.hadoop:maprfs:jar:5.2.1-mapr
com.mapr.fs:mapr-hbase:jar:5.2.1-mapr

{noformat}
[INFO] org.apache.drill.contrib:drill-format-mapr:jar:1.12.0-SNAPSHOT
[INFO] +- com.mapr.hadoop:maprfs:jar:5.2.1-mapr:compile
[INFO] |  \- org.json:json:jar:20080701:compile
[INFO] \- com.mapr.fs:mapr-hbase:jar:5.2.1-mapr:compile
[INFO]\- (org.json:json:jar:20080701:compile - omitted for duplicate)
{noformat}

Need to make sure we won't have any dependencies from these libs to json.org 
lib and ban this lib in main pom.xml file.

Issue is critical since Apache release won't happen until we make sure json.org 
lib is not used (https://www.apache.org/legal/resolved.html).



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226962#comment-16226962
 ] 

ASF GitHub Bot commented on DRILL-5822:
---

Github user vdiravka commented on the issue:

https://github.com/apache/drill/pull/1017
  
@paul-rogers Could you please review this PR? You can find a short 
description here or more detailed - in the jira ticket 
[DRILL-5822](https://issues.apache.org/jira/browse/DRILL-5822).


> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-3993) Rebase Drill on Calcite master branch

2017-10-31 Thread Roman Kulyk (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-3993?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226880#comment-16226880
 ] 

Roman Kulyk commented on DRILL-3993:


Still working on the java-exec test module. There are 18 tests left. You can 
see the list of errors below:
{code}
TestFunctionsWithTypeExpoQueries.testEqualBetweenIntervalAndTimestampDiff:403->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestUnionDistinct.testDiffDataTypesAndModes:288->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestExampleQueries.testDRILL_3004:1036->BaseTestQuery.testRunAndReturn:360 » Rpc
TestExampleQueries.testFilterInSubqueryAndOutside » UserRemote DATA_READ 
ERROR...
TestExampleQueries.testDateImplicitCasting:1205->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestCaseNullableTypes.testCaseNullableTypesVarchar:45->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestInfoSchema.describeTableWithSchemaAndColQualifier:271->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestInfoSchema.describeTableWithColQualifier:258->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestSqlBracketlessSyntax.checkComplexExpressionParsing:54 » NoClassDefFound 
co...
TestNestedLoopJoin.testNLJWithEmptyBatch:229->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestFlatten.testFlattenOnEmptyArrayAndNestedMap:600->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestParquetWriter.testWriteDecimal:591 »  After matching 0 records, did not 
fi...
TestCastEmptyStrings.testCastInputTypeNonNullableVarCharToDecimal » UserRemote
TestCastEmptyStrings.testCastInputTypeNonNullableVarCharToNumeric » UserRemote
TestDateTruncFunctions.dateTruncOnIntervalDay:301->BaseTestQuery.testRunAndReturn:360
 » Rpc
TestFunctionsQuery.testDecimalRoundUp:417 »  After matching 0 records, did 
not...
TestFunctionsQuery.testRandom:922 »  After matching 0 records, did not find 
ex...
TestUnionAll.testDiffDataTypesAndModes:272->BaseTestQuery.testRunAndReturn:360 
» Rpc
{code}

> Rebase Drill on Calcite master branch
> -
>
> Key: DRILL-3993
> URL: https://issues.apache.org/jira/browse/DRILL-3993
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Query Planning & Optimization
>Affects Versions: 1.2.0
>Reporter: Sudheesh Katkam
>Assignee: Roman Kulyk
>
> Calcite keeps moving, and now we need to catch up to Calcite 1.5, and ensure 
> there are no regressions.
> Also, how do we resolve this 'catching up' issue in the long term?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Created] (DRILL-5916) Drill document window function example on LAST_VALUE is incorrect

2017-10-31 Thread Raymond Wong (JIRA)
Raymond Wong created DRILL-5916:
---

 Summary: Drill document window function example on LAST_VALUE is 
incorrect
 Key: DRILL-5916
 URL: https://issues.apache.org/jira/browse/DRILL-5916
 Project: Apache Drill
  Issue Type: Bug
  Components: Documentation
Affects Versions: 1.11.0
Reporter: Raymond Wong
Priority: Minor


The top and bottom review count example query result is showing incorrect 
values for the LAST_VALUE column. 
([https://drill.apache.org/docs/analyzing-data-using-window-functions/] )

The LAST_VALUE column should have the same value as the review count of each 
row because the default Window Frame is RANGE BETWEEN UNBOUNDED PRECEDING AND 
*CURRENT ROW*.

Query result using 2017 yelp data set.

{quote}
SELECT name, city, review_count,
  FIRST_VALUE(review_count)
OVER(PARTITION BY city ORDER BY review_count DESC) AS top_review_count,
  LAST_VALUE(review_count)
OVER(PARTITION BY city ORDER BY review_count DESC) AS bottom_review_count
FROM dfs.yelp.`yelp_academic_dataset_business.json`
LIMIT 30

||name  ||city   
||review_count ||top_review_count||bottom_review_count ||
|Lululemon Athletica   ||5  
  |5|5   |
|Aberdour Castle   |Aberdour|4  
  |4|4   |
|Cupz N' Crepes|Ahwatukee   |236
  |236  |236 |
|My Wine Cellar|Ahwatukee   |158
  |236  |158 |
|Florencia Pizza Bistro|Ahwatukee   |129
  |236  |129 |
|Barro's Pizza |Ahwatukee   |62 
  |236  |62  |
|Kathy's Alterations   |Ahwatukee   |30 
  |236  |30  |
|Hertz Rent A Car  |Ahwatukee   |26 
  |236  |26  |
|Active Kids Pediatrics|Ahwatukee   |18 
  |236  |18  |
|Dental by Design  |Ahwatukee   |18 
  |236  |18  |
|Desert Dog Pet Care   |Ahwatukee   |10 
  |236  |10  |
|McDonald's|Ahwatukee   |7  
  |236  |7   |
|U-Haul|Ahwatukee   |6  
  |236  |6   |
|Sprinkler Detective   |Ahwatukee   |5  
  |236  |5   |
|Hi-Health |Ahwatukee   |4  
  |236  |4   |
|Healthy and Clean Living Environments |Ahwatukee   |4  
  |236  |4   |
|Designs By Christa|Ahwatukee   |4  
  |236  |4   |
{quote}

Changing the LAST_VAULE's Window Frame to RANGE BETWEEN UNBOUNDED PRECEDING AND 
*UNBOUNDED FOLLOWING*.

{quote}
SELECT name, city, review_count,
  FIRST_VALUE(review_count)
OVER(PARTITION BY city ORDER BY review_count DESC) AS top_review_count,
  LAST_VALUE(review_count)
OVER(PARTITION BY city ORDER BY review_count DESC RANGE BETWEEN UNBOUNDED 
PRECEDING AND UNBOUNDED FOLLOWING) AS bottom_review_count
FROM dfs.yelp.`yelp_academic_dataset_business.json`
LIMIT 30
;

||name  ||city
||review_count ||top_review_count ||bottom_review_count ||
|Lululemon Athletica   ||5  
  |5|5   |
|Aberdour Castle   |Aberdour|4  
  |4|4   |
|Cupz N' Crepes|Ahwatukee   |236
  |236  |4   |
|My Wine Cellar|Ahwatukee   |158
  |236  |4   |
|Florencia Pizza Bistro|Ahwatukee   |129
  |236  |4   |
|Barro's Pizza |Ahwatukee   |62 
  |236  |4   |
|Kathy's Alterations   |Ahwatukee   |30 
  |236  |4   |
|Hertz Rent A Car  |Ahwatukee   |26 
  |236  |4   

[jira] [Commented] (DRILL-5717) change some date time unit cases with specific timezone or Local

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226856#comment-16226856
 ] 

ASF GitHub Bot commented on DRILL-5717:
---

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

https://github.com/apache/drill/pull/904#discussion_r148000718
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/testing/TestDateConversions.java
 ---
@@ -117,10 +130,18 @@ public void testJodaTime() throws Exception {
   .baselineValues(true, true)
   .baselineValues(false, true)
   .go();
+
--- End diff --

Please revert the addition of empty line.


> change some date time unit cases with specific timezone or Local
> 
>
> Key: DRILL-5717
> URL: https://issues.apache.org/jira/browse/DRILL-5717
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Tools, Build & Test
>Affects Versions: 1.9.0, 1.11.0
>Reporter: weijie.tong
>
> Some date time test cases like  JodaDateValidatorTest  is not Local 
> independent .This will cause other Local's users's test phase to fail. We 
> should let these test cases to be Local env independent.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5717) change some date time unit cases with specific timezone or Local

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226852#comment-16226852
 ] 

ASF GitHub Bot commented on DRILL-5717:
---

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

https://github.com/apache/drill/pull/904#discussion_r147996953
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java
 ---
@@ -26,14 +26,14 @@
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.sql.Date;
 
 @Category({UnlikelyTest.class, SqlFunctionTest.class})
 public class TestNewDateFunctions extends BaseTestQuery {
   DateTime date;
   DateTimeFormatter formatter;
   long unixTimeStamp = -1;
 
+
--- End diff --

Please revert the addition of empty line there.


> change some date time unit cases with specific timezone or Local
> 
>
> Key: DRILL-5717
> URL: https://issues.apache.org/jira/browse/DRILL-5717
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Tools, Build & Test
>Affects Versions: 1.9.0, 1.11.0
>Reporter: weijie.tong
>
> Some date time test cases like  JodaDateValidatorTest  is not Local 
> independent .This will cause other Local's users's test phase to fail. We 
> should let these test cases to be Local env independent.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5717) change some date time unit cases with specific timezone or Local

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226855#comment-16226855
 ] 

ASF GitHub Bot commented on DRILL-5717:
---

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

https://github.com/apache/drill/pull/904#discussion_r147997514
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewDateFunctions.java
 ---
@@ -26,14 +26,14 @@
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.sql.Date;
 
--- End diff --

When you removed unused import, two empty lines stayed. Could you please 
also remove one of them?


> change some date time unit cases with specific timezone or Local
> 
>
> Key: DRILL-5717
> URL: https://issues.apache.org/jira/browse/DRILL-5717
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Tools, Build & Test
>Affects Versions: 1.9.0, 1.11.0
>Reporter: weijie.tong
>
> Some date time test cases like  JodaDateValidatorTest  is not Local 
> independent .This will cause other Local's users's test phase to fail. We 
> should let these test cases to be Local env independent.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5717) change some date time unit cases with specific timezone or Local

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226853#comment-16226853
 ] 

ASF GitHub Bot commented on DRILL-5717:
---

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

https://github.com/apache/drill/pull/904#discussion_r148002267
  
--- Diff: 
logical/src/test/java/org/apache/drill/common/expression/fn/JodaDateValidatorTest.java
 ---
@@ -119,13 +118,14 @@ public void testDateDayOfYearYearFormat() {
 
   @Test
   public void testTimeHoursMinutesSecondsFormat() {
+Locale.setDefault(Locale.US);
--- End diff --

Not sure that this change is necessary.


> change some date time unit cases with specific timezone or Local
> 
>
> Key: DRILL-5717
> URL: https://issues.apache.org/jira/browse/DRILL-5717
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Tools, Build & Test
>Affects Versions: 1.9.0, 1.11.0
>Reporter: weijie.tong
>
> Some date time test cases like  JodaDateValidatorTest  is not Local 
> independent .This will cause other Local's users's test phase to fail. We 
> should let these test cases to be Local env independent.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226679#comment-16226679
 ] 

Vitalii Diravka edited comment on DRILL-5822 at 10/31/17 1:37 PM:
--

This is an old topic which was discussed in DRILL-1499 and DRILL-3101. 
For now there is no need to canonicalize the batch or container since 
RecordBatchLoader swallows the "schema change" if two batches have different 
column ordering. That's why DRILL-847 is oudated.
PR for this ticket - https://github.com/apache/drill/pull/1017


was (Author: vitalii):
This is an old topic which was discussed in DRILL-1499 and DRILL-3101. 
For now there is no need to canonicalize the batch or container since 
RecordBatchLoader swallows the "schema change" if two batches have different 
column ordering. That's why DRILL-847 is oudated.

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5717) change some date time unit cases with specific timezone or Local

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5717?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226693#comment-16226693
 ] 

ASF GitHub Bot commented on DRILL-5717:
---

Github user weijietong commented on the issue:

https://github.com/apache/drill/pull/904
  
@vvysotskyi  thanks for your help! have done, please review.


> change some date time unit cases with specific timezone or Local
> 
>
> Key: DRILL-5717
> URL: https://issues.apache.org/jira/browse/DRILL-5717
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Tools, Build & Test
>Affects Versions: 1.9.0, 1.11.0
>Reporter: weijie.tong
>
> Some date time test cases like  JodaDateValidatorTest  is not Local 
> independent .This will cause other Local's users's test phase to fail. We 
> should let these test cases to be Local env independent.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226679#comment-16226679
 ] 

Vitalii Diravka commented on DRILL-5822:


This is an old topic which was discussed in DRILL-1499 and DRILL-3101. 
For now there is no need to canonicalize the batch or container since 
RecordBatchLoader swallows the "schema change" if two batches have different 
column ordering. That's why DRILL-847 is oudated.

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka updated DRILL-5822:
---
Labels:   (was: read)

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka updated DRILL-5822:
---
Component/s: (was: Storage - JSON)

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
>  Labels: read
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka updated DRILL-5822:
---
Labels: read  (was: )

> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
>  Labels: read
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple fragments.
> Repro steps:
> 1) {code}alter session set `planner.slice_target`=1;{code}
> 2) ORDER BY clause in the query.
> Scenarios:
> {code}
> 0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.082 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> | n_nationkey  |  n_name  | n_regionkey  |  n_comment 
>   |
> +--+--+--+--+
> | 0| ALGERIA  | 0|  haggle. carefully final deposits 
> detect slyly agai  |
> +--+--+--+--+
> 1 row selected (0.141 seconds)
> 0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
> +---++
> |  ok   |summary |
> +---++
> | true  | planner.slice_target updated.  |
> +---++
> 1 row selected (0.091 seconds)
> 0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by 
> n_name limit 1;
> +--+--+--+--+
> |  n_comment   |  n_name  | 
> n_nationkey  | n_regionkey  |
> +--+--+--+--+
> |  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0 
>| 0|
> +--+--+--+--+
> 1 row selected (0.201 seconds)
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-847) Merging Receiver requires to canonicalize schemas of input batches

2017-10-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226669#comment-16226669
 ] 

ASF GitHub Bot commented on DRILL-847:
--

GitHub user vdiravka opened a pull request:

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

The query with "SELECT *" with "ORDER BY" clause and `planner.slice_t…

…arget`=1 doesn't preserve column order

Issue: Columns ordering doesn't preserve for the star query with sorting 
when this is planned into multiple fragments.
Solution: The commit for DRILL-847 is oudated. There is no need to 
canonicalize the batch or container since RecordBatchLoader swallows the 
"schema change" for now if two batches have different column ordering.

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

$ git pull https://github.com/vdiravka/drill DRILL-5822

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

https://github.com/apache/drill/pull/1017.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 #1017


commit 922505c35ed18f5cf76a5ded77a734827c15eb59
Author: Vitalii Diravka 
Date:   2017-10-26T18:07:33Z

The query with "SELECT *" with "ORDER BY" clause and 
`planner.slice_target`=1 doesn't preserve column order

- The commit for DRILL-847 is oudated. There is no need to canonicalize the 
batch or container since RecordBatchLoader
  swallows the "schema change" for now if two batches have different column 
ordering.




> Merging Receiver requires to canonicalize schemas of input batches 
> ---
>
> Key: DRILL-847
> URL: https://issues.apache.org/jira/browse/DRILL-847
> Project: Apache Drill
>  Issue Type: Bug
>Reporter: Jinfeng Ni
>Assignee: Jinfeng Ni
> Fix For: 0.4.0
>
> Attachments: DRILL-847.1.patch
>
>
> The Merge Receivers assumes all the incoming batches have the same schemas. 
> If the incoming batches happen to have different schemas, the run-time 
> generated code for merge receiver would not be able to handle the different 
> schemas, and could throw ClassCast Exception.
> To fix it, we need canonicalize the schemas of incoming batches for merge 
> receiver. If the schemas are still different after this step,  Merge Receiver 
> would throw exception. 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Updated] (DRILL-5822) The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 doesn't preserve column order

2017-10-31 Thread Vitalii Diravka (JIRA)

 [ 
https://issues.apache.org/jira/browse/DRILL-5822?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vitalii Diravka updated DRILL-5822:
---
Description: 
Columns ordering doesn't preserve for the star query with sorting when this is 
planned into multiple fragments.
Repro steps:
1) {code}alter session set `planner.slice_target`=1;{code}
2) ORDER BY clause in the query.

Scenarios:
{code}
0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
+---++
|  ok   |summary |
+---++
| true  | planner.slice_target updated.  |
+---++
1 row selected (0.082 seconds)
0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by n_name 
limit 1;
+--+--+--+--+
| n_nationkey  |  n_name  | n_regionkey  |  n_comment   
|
+--+--+--+--+
| 0| ALGERIA  | 0|  haggle. carefully final deposits 
detect slyly agai  |
+--+--+--+--+
1 row selected (0.141 seconds)
0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
+---++
|  ok   |summary |
+---++
| true  | planner.slice_target updated.  |
+---++
1 row selected (0.091 seconds)
0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by n_name 
limit 1;
+--+--+--+--+
|  n_comment   |  n_name  | n_nationkey 
 | n_regionkey  |
+--+--+--+--+
|  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0   
 | 0|
+--+--+--+--+
1 row selected (0.201 seconds)
{code}

  was:
Repro steps:
1) {code}alter session set `planner.slice_target`=1;{code}
2) ORDER BY clause in the query.

Scenarios:
{code}
0: jdbc:drill:zk=local> alter session reset `planner.slice_target`;
+---++
|  ok   |summary |
+---++
| true  | planner.slice_target updated.  |
+---++
1 row selected (0.082 seconds)
0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by n_name 
limit 1;
+--+--+--+--+
| n_nationkey  |  n_name  | n_regionkey  |  n_comment   
|
+--+--+--+--+
| 0| ALGERIA  | 0|  haggle. carefully final deposits 
detect slyly agai  |
+--+--+--+--+
1 row selected (0.141 seconds)
0: jdbc:drill:zk=local> alter session set `planner.slice_target`=1;
+---++
|  ok   |summary |
+---++
| true  | planner.slice_target updated.  |
+---++
1 row selected (0.091 seconds)
0: jdbc:drill:zk=local> select * from cp.`tpch/nation.parquet` order by n_name 
limit 1;
+--+--+--+--+
|  n_comment   |  n_name  | n_nationkey 
 | n_regionkey  |
+--+--+--+--+
|  haggle. carefully final deposits detect slyly agai  | ALGERIA  | 0   
 | 0|
+--+--+--+--+
1 row selected (0.201 seconds)
{code}


> The query with "SELECT *" with "ORDER BY" clause and `planner.slice_target`=1 
> doesn't preserve column order
> ---
>
> Key: DRILL-5822
> URL: https://issues.apache.org/jira/browse/DRILL-5822
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Storage - JSON
>Affects Versions: 1.11.0
>Reporter: Prasad Nagaraj Subramanya
>Assignee: Vitalii Diravka
> Fix For: 1.12.0
>
>
> Columns ordering doesn't preserve for the star query with sorting when this 
> is planned into multiple 

[jira] [Commented] (DRILL-5895) Fix unit tests for mongo storage plugin

2017-10-31 Thread Volodymyr Tkach (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5895?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226544#comment-16226544
 ] 

Volodymyr Tkach commented on DRILL-5895:


Merged into master with commit id c8d01b3141b0b755d4311340c3e5823214332c70

> Fix unit tests for mongo storage plugin
> ---
>
> Key: DRILL-5895
> URL: https://issues.apache.org/jira/browse/DRILL-5895
> Project: Apache Drill
>  Issue Type: Bug
>  Components: Storage - MongoDB
>Affects Versions: 1.11.0
>Reporter: Volodymyr Tkach
>Assignee: Volodymyr Tkach
>  Labels: ready-to-commit
> Fix For: 1.12.0
>
>
> Mongo tests finish with following exception intermittently. It happens 
> because  [timeout 
> value|https://github.com/flapdoodle-oss/de.flapdoodle.embed.process/blob/1.7/src/main/java/de/flapdoodle/embed/process/runtime/ProcessControl.java#L132]
>  from   de.flapdoodle.embed.process library is too low for the mongod process 
> to be stopped gracefully. 
> I have created 
> [issue|https://github.com/flapdoodle-oss/de.flapdoodle.embed.process/issues/64]
>  to suggest making timeout configurable. For now as temporary solution we 
> will log mongod exception instead of throwing it.
> {noformat}
> [mongod output] Exception in thread "Thread-25" 
> java.lang.IllegalStateException: Couldn't kill mongod process!
> 
> Something bad happend. We couldn't kill mongod process, and tried a lot.
> If you want this problem solved you can help us if you open a new issue.
> Follow this link:
> https://github.com/flapdoodle-oss/de.flapdoodle.embed.mongo/issues
> Thank you:)
> 
>   at 
> de.flapdoodle.embed.process.runtime.ProcessControl.waitForProcessGotKilled(ProcessControl.java:192)
>   at 
> de.flapdoodle.embed.process.runtime.ProcessControl.stop(ProcessControl.java:76)
>   at 
> de.flapdoodle.embed.process.runtime.AbstractProcess.stopProcess(AbstractProcess.java:189)
>   at 
> de.flapdoodle.embed.mongo.AbstractMongoProcess.stopInternal(AbstractMongoProcess.java:117)
>   at 
> de.flapdoodle.embed.process.runtime.AbstractProcess.stop(AbstractProcess.java:170)
>   at 
> de.flapdoodle.embed.process.runtime.Executable.stop(Executable.java:73)
>   at 
> de.flapdoodle.embed.process.runtime.Executable$JobKiller.run(Executable.java:90)
>   at java.lang.Thread.run(Thread.java:745)
> Exception in thread "Thread-13" java.lang.IllegalStateException: Couldn't 
> kill mongod process!
> 
> Something bad happend. We couldn't kill mongod process, and tried a lot.
> If you want this problem solved you can help us if you open a new issue.
> Follow this link:
> https://github.com/flapdoodle-oss/de.flapdoodle.embed.mongo/issues
> Thank you:)
> 
>   at 
> de.flapdoodle.embed.process.runtime.ProcessControl.waitForProcessGotKilled(ProcessControl.java:192)
>   at 
> de.flapdoodle.embed.process.runtime.ProcessControl.stop(ProcessControl.java:76)
>   at 
> de.flapdoodle.embed.process.runtime.AbstractProcess.stopProcess(AbstractProcess.java:189)
>   at 
> de.flapdoodle.embed.mongo.AbstractMongoProcess.stopInternal(AbstractMongoProcess.java:117)
>   at 
> de.flapdoodle.embed.process.runtime.AbstractProcess.stop(AbstractProcess.java:170)
>   at 
> de.flapdoodle.embed.process.runtime.Executable.stop(Executable.java:73)
>   at 
> de.flapdoodle.embed.process.runtime.Executable$JobKiller.run(Executable.java:90)
>   at java.lang.Thread.run(Thread.java:745)
> Results :
> Tests in error: 
>   MongoTestSuit.tearDownCluster:260 » IllegalState Couldn't kill mongod 
> process!...
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Commented] (DRILL-5910) Logging exception when custom AuthenticatorFactory not found

2017-10-31 Thread Volodymyr Tkach (JIRA)

[ 
https://issues.apache.org/jira/browse/DRILL-5910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16226540#comment-16226540
 ] 

Volodymyr Tkach commented on DRILL-5910:


Merge into master with commit id b0b7df10da14ab94cd657f0c1e3a0c3ff242bedd

> Logging exception when  custom AuthenticatorFactory not found
> -
>
> Key: DRILL-5910
> URL: https://issues.apache.org/jira/browse/DRILL-5910
> Project: Apache Drill
>  Issue Type: Improvement
>Affects Versions: 1.11.0
>Reporter: Volodymyr Tkach
>Assignee: Volodymyr Tkach
>Priority: Minor
>  Labels: ready-to-commit
> Fix For: 1.12.0
>
>
> We need to log the exception when any of custom AuthenticatorFactory fails to 
> be instansiated in ClientAuthenticatorProvider constructor. We are doing this 
> to allow drill to use other available AuthenticatorFactory
> Steps to repoduce:
> 1) Configure plain authentication
> 2) Add 
> -Ddrill.customAuthFactories=org.apache.drill.exec.rpc.security.maprsasl.MapRSaslFactory
>  to SQLLINE_JAVA_OPTS or another class that is not present in classpath.
> 3) Run sqlline and connect to drillbit



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)