[
https://issues.apache.org/jira/browse/TAJO-1565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14501837#comment-14501837
]
ASF GitHub Bot commented on TAJO-1565:
--------------------------------------
Github user dongjoon-hyun commented on the pull request:
https://github.com/apache/tajo/pull/543#issuecomment-94274754
Hi, @jihoonson . I found a related description in PostgresSQL documentation.
http://www.postgresql.org/docs/9.4/static/queries-order.html
*After a query has produced an output table (after the select list has
been processed) it can optionally be sorted*. If sorting is not chosen, the
rows will be returned in an unspecified order. The actual order in that case
will depend on the scan and join plan types and the order on disk, but it must
not be relied on. A particular output ordering can only be guaranteed if the
sort step is explicitly chosen.
> Simple column name in ORDER BY is failed with "Ambiguous Column Name ERROR".
> ----------------------------------------------------------------------------
>
> Key: TAJO-1565
> URL: https://issues.apache.org/jira/browse/TAJO-1565
> Project: Tajo
> Issue Type: Bug
> Affects Versions: 0.10.0
> Reporter: Hyoungjun Kim
> Assignee: Dongjoon Hyun
> Attachments: TAJO-1565.Hyun.150419.0.patch.txt
>
>
> I ran the following query which contains self join and doesn't use a fully
> qualified column name in ORDER BY clause. The query failed with "Ambiguous
> Column Name ERROR".
> {code:sql}
> select n1.n_nationkey, n2.n_name
> from nation n1, nation n2, nation n3
> where n1.n_nationkey = n2.n_nationkey
> and n1.n_nationkey = n3.n_nationkey
> order by n_name;
> {code}
> {noformat}
> org.apache.tajo.plan.algebra.AmbiguousFieldException: ERROR: column name
> Ambiguous Column Name: default.n1.n_name (TEXT), default.n2.n_name (TEXT),
> default.n3.n_name (TEXT) is ambiguous
> at
> org.apache.tajo.plan.nameresolver.NameResolver.ensureUniqueColumn(NameResolver.java:286)
> at
> org.apache.tajo.plan.nameresolver.NameResolver.resolveFromAllRelsInBlock(NameResolver.java:183)
> at
> org.apache.tajo.plan.nameresolver.ResolverByLegacy.resolveColumnWithoutQualifier(ResolverByLegacy.java:104)
> at
> org.apache.tajo.plan.nameresolver.ResolverByLegacy.resolve(ResolverByLegacy.java:44)
> at
> org.apache.tajo.plan.nameresolver.NameResolver.resolve(NameResolver.java:90)
> at
> org.apache.tajo.plan.ExprNormalizer.visitColumnReference(ExprNormalizer.java:344)
> at
> org.apache.tajo.plan.ExprNormalizer.visitColumnReference(ExprNormalizer.java:84)
> at
> org.apache.tajo.plan.algebra.BaseAlgebraVisitor.visit(BaseAlgebraVisitor.java:218)
> at
> org.apache.tajo.plan.visitor.SimpleAlgebraVisitor.visit(SimpleAlgebraVisitor.java:43)
> at
> org.apache.tajo.plan.ExprNormalizer.normalize(ExprNormalizer.java:122)
> at
> org.apache.tajo.plan.ExprNormalizer.normalize(ExprNormalizer.java:115)
> at
> org.apache.tajo.plan.LogicalPlanner.visitSort(LogicalPlanner.java:829)
> at org.apache.tajo.plan.LogicalPlanner.visitSort(LogicalPlanner.java:66)
> at
> org.apache.tajo.plan.algebra.BaseAlgebraVisitor.visit(BaseAlgebraVisitor.java:65)
> at
> org.apache.tajo.plan.LogicalPlanner.visitProjection(LogicalPlanner.java:238)
> at
> org.apache.tajo.plan.LogicalPlanner.visitProjection(LogicalPlanner.java:66)
> at
> org.apache.tajo.plan.algebra.BaseAlgebraVisitor.visit(BaseAlgebraVisitor.java:59)
> at
> org.apache.tajo.plan.LogicalPlanner.createPlan(LogicalPlanner.java:143)
> at
> org.apache.tajo.plan.LogicalPlanner.createPlan(LogicalPlanner.java:131)
> at
> org.apache.tajo.master.GlobalEngine.createLogicalPlan(GlobalEngine.java:271)
> at
> org.apache.tajo.master.GlobalEngine.executeQuery(GlobalEngine.java:188)
> at
> org.apache.tajo.master.TajoMasterClientService$TajoMasterClientProtocolServiceHandler.submitQuery(TajoMasterClientService.java:287)
> at
> org.apache.tajo.ipc.TajoMasterClientProtocol$TajoMasterClientProtocolService$2.callBlockingMethod(TajoMasterClientProtocol.java:543)
> at
> org.apache.tajo.rpc.BlockingRpcServer$ServerHandler.channelRead(BlockingRpcServer.java:113)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
> at
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
> at
> io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:182)
> at
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
> at
> io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
> at
> io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
> at
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130)
> at
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
> at
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
> at
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
> at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
> at
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116)
> at java.lang.Thread.run(Thread.java:744)
> {noformat}
> This query run successfully on PostgreSQL.
> {noformat}
> babokim=# select n1.n_nationkey, n2.n_name
> babokim-# from nation n1, nation n2, nation n3
> babokim-# where n1.n_nationkey = n2.n_nationkey
> babokim-# and n1.n_nationkey = n3.n_nationkey
> babokim-# order by n_name;;
> n_nationkey | n_name
> -------------+-----------
> 0 | ALGERIA
> 1 | ARGENTINA
> 2 | BRAZIL
> (3 rows)
> {noformat}
> Also I found a strange code in NameResolver.ensureUniqueColumn().
> I think "else if (candidates.size() > 2)" statement should be "else". I don't
> know why compare great than 2.
> {code:java}
> static Column ensureUniqueColumn(List<Column> candidates) throws
> VerifyException {
> if (candidates.size() == 1) {
> return candidates.get(0);
> } else if (candidates.size() > 2) {
> StringBuilder sb = new StringBuilder();
> boolean first = true;
> for (Column column : candidates) {
> if (first) {
> first = false;
> } else {
> sb.append(", ");
> }
> sb.append(column);
> }
> throw new AmbiguousFieldException("Ambiguous Column Name: " +
> sb.toString());
> } else {
> return null;
> }
> }
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)