You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2015/06/18 09:31:14 UTC

[01/13] incubator-kylin git commit: KYLIN-838

Repository: incubator-kylin
Updated Branches:
  refs/heads/KYLIN-780 241555359 -> fcb204467


KYLIN-838


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/29bb731d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/29bb731d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/29bb731d

Branch: refs/heads/KYLIN-780
Commit: 29bb731d4f613acbaaa6cd960e1c7809271719ee
Parents: 96db2de
Author: qianhao.zhou <qi...@ebay.com>
Authored: Thu Jun 18 14:29:16 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Thu Jun 18 14:29:16 2015 +0800

----------------------------------------------------------------------
 .../common/persistence/FileResourceStore.java   | 35 +++++++--
 .../common/persistence/HBaseResourceStore.java  | 82 +++++++++++++-------
 .../kylin/common/persistence/RawResource.java   | 16 ++++
 .../kylin/common/persistence/ResourceStore.java | 24 ++++++
 .../org/apache/kylin/job/dao/ExecutableDao.java | 26 +++----
 .../kylin/job/execution/AbstractExecutable.java | 41 ++++++----
 .../kylin/job/manager/ExecutableManager.java    | 58 ++++++++------
 .../apache/kylin/rest/service/BasicService.java | 18 +++--
 .../apache/kylin/rest/service/JobService.java   | 54 +++++++++----
 9 files changed, 248 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
index 33f5841..38c6d9d 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/FileResourceStore.java
@@ -18,16 +18,14 @@
 
 package org.apache.kylin.common.persistence;
 
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
+import java.io.*;
 import java.util.ArrayList;
+import java.util.List;
 
+import com.google.common.collect.Lists;
 import org.apache.commons.io.IOUtils;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 
 public class FileResourceStore extends ResourceStore {
@@ -63,6 +61,31 @@ public class FileResourceStore extends ResourceStore {
     }
 
     @Override
+    protected List<RawResource> getAllResources(String rangeStart, String rangeEnd) throws IOException {
+        List<RawResource> result = Lists.newArrayList();
+        try {
+            String commonPrefix = StringUtils.getCommonPrefix(rangeEnd, rangeStart);
+            commonPrefix = commonPrefix.substring(0, commonPrefix.lastIndexOf("/") + 1);
+            final ArrayList<String> resources = listResourcesImpl(commonPrefix);
+            for (String resource : resources) {
+                if (resource.compareTo(rangeStart) >= 0 && resource.compareTo(rangeEnd) <= 0) {
+                    if (existsImpl(resource)) {
+                        result.add(new RawResource(getResourceImpl(resource), getResourceTimestampImpl(resource)));
+                    }
+                }
+            }
+            return result;
+        } catch (IOException ex) {
+            for (RawResource rawResource : result) {
+                IOUtils.closeQuietly(rawResource.resource);
+            }
+            throw ex;
+        } catch (Exception ex) {
+            throw new UnsupportedOperationException(ex);
+        }
+    }
+
+    @Override
     protected InputStream getResourceImpl(String resPath) throws IOException {
         File f = file(resPath);
         if (f.exists() && f.isFile())

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
index 2868368..6c5847e 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/HBaseResourceStore.java
@@ -18,34 +18,22 @@
 
 package org.apache.kylin.common.persistence;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
+import com.google.common.collect.Lists;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HConnection;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.*;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
-import org.apache.kylin.common.util.Bytes;
-
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.BytesUtil;
 import org.apache.kylin.common.util.HadoopUtil;
 
+import java.io.*;
+import java.util.*;
+
 public class HBaseResourceStore extends ResourceStore {
 
     private static final String DEFAULT_TABLE_NAME = "kylin_metadata";
@@ -144,11 +132,36 @@ public class HBaseResourceStore extends ResourceStore {
     }
 
     @Override
-    protected InputStream getResourceImpl(String resPath) throws IOException {
-        Result r = getByScan(resPath, B_FAMILY, B_COLUMN);
-        if (r == null)
-            return null;
+    protected List<RawResource> getAllResources(String rangeStart, String rangeEnd) throws IOException {
+        byte[] startRow = Bytes.toBytes(rangeStart);
+        byte[] endRow = plusZero(Bytes.toBytes(rangeEnd));
+
+        Scan scan = new Scan(startRow, endRow);
+        scan.addColumn(B_FAMILY, B_COLUMN_TS);
+        scan.addColumn(B_FAMILY, B_COLUMN);
 
+        HTableInterface table = getConnection().getTable(getAllInOneTableName());
+        List<RawResource> result = Lists.newArrayList();
+        try {
+            ResultScanner scanner = table.getScanner(scan);
+            for (Result r : scanner) {
+                result.add(new RawResource(getInputStream(Bytes.toString(r.getRow()), r), getTimestamp(r)));
+            }
+        } catch (IOException e) {
+            for (RawResource rawResource : result) {
+                IOUtils.closeQuietly(rawResource.resource);
+            }
+            throw e;
+        } finally {
+            IOUtils.closeQuietly(table);
+        }
+        return result;
+    }
+
+    private InputStream getInputStream(String resPath, Result r) throws IOException {
+        if (r == null) {
+            return null;
+        }
         byte[] value = r.getValue(B_FAMILY, B_COLUMN);
         if (value.length == 0) {
             Path redirectPath = bigCellHDFSPath(resPath);
@@ -161,13 +174,24 @@ public class HBaseResourceStore extends ResourceStore {
         }
     }
 
+    private long getTimestamp(Result r) {
+        if (r == null) {
+            return 0;
+        } else {
+            return Bytes.toLong(r.getValue(B_FAMILY, B_COLUMN_TS));
+        }
+    }
+
+    @Override
+    protected InputStream getResourceImpl(String resPath) throws IOException {
+        Result r = getByScan(resPath, B_FAMILY, B_COLUMN);
+        return getInputStream(resPath, r);
+    }
+
     @Override
     protected long getResourceTimestampImpl(String resPath) throws IOException {
         Result r = getByScan(resPath, B_FAMILY, B_COLUMN_TS);
-        if (r == null)
-            return 0;
-        else
-            return Bytes.toLong(r.getValue(B_FAMILY, B_COLUMN_TS));
+        return getTimestamp(r);
     }
 
     @Override
@@ -197,8 +221,10 @@ public class HBaseResourceStore extends ResourceStore {
             Put put = buildPut(resPath, newTS, row, content, table);
 
             boolean ok = table.checkAndPut(row, B_FAMILY, B_COLUMN_TS, bOldTS, put);
-            if (!ok)
-                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + getResourceTimestamp(resPath));
+            if (!ok) {
+                long real = getResourceTimestamp(resPath);
+                throw new IllegalStateException("Overwriting conflict " + resPath + ", expect old TS " + oldTS + ", but it is " + real);
+            }
 
             table.flushCommits();
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java b/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
new file mode 100644
index 0000000..2c3238f
--- /dev/null
+++ b/common/src/main/java/org/apache/kylin/common/persistence/RawResource.java
@@ -0,0 +1,16 @@
+package org.apache.kylin.common.persistence;
+
+import java.io.InputStream;
+
+/**
+ */
+public class RawResource {
+
+    public final InputStream resource;
+    public final long timestamp;
+
+    public RawResource(InputStream resource, long timestamp) {
+        this.resource = resource;
+        this.timestamp = timestamp;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index c5db1a5..7021915 100644
--- a/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -24,6 +24,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -51,6 +52,7 @@ abstract public class ResourceStore {
     public static final String TABLE_EXD_RESOURCE_ROOT = "/table_exd";
     public static final String TABLE_RESOURCE_ROOT = "/table";
 
+
     private static ConcurrentHashMap<KylinConfig, ResourceStore> CACHE = new ConcurrentHashMap<KylinConfig, ResourceStore>();
 
     public static final ArrayList<Class<? extends ResourceStore>> knownImpl = new ArrayList<Class<? extends ResourceStore>>();
@@ -144,6 +146,28 @@ abstract public class ResourceStore {
         return getResourceImpl(norm(resPath));
     }
 
+    final public <T extends RootPersistentEntity> List<T> getAllResources(String rangeStart, String rangeEnd, Class<T> clazz, Serializer<T> serializer) throws IOException {
+        final List<RawResource> allResources = getAllResources(rangeStart, rangeEnd);
+        if (allResources.isEmpty()) {
+            return Collections.emptyList();
+        }
+        List<T> result = Lists.newArrayList();
+        try {
+            for (RawResource rawResource : allResources) {
+                final T element = serializer.deserialize(new DataInputStream(rawResource.resource));
+                element.setLastModified(rawResource.timestamp);
+                result.add(element);
+            }
+            return result;
+        } finally {
+            for (RawResource rawResource : allResources) {
+                IOUtils.closeQuietly(rawResource.resource);
+            }
+        }
+    }
+
+    abstract protected List<RawResource> getAllResources(String rangeStart, String rangeEnd) throws IOException;
+
     abstract protected InputStream getResourceImpl(String resPath) throws IOException;
 
     final public long getResourceTimestamp(String resPath) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
index 305d570..d7ee8b3 100644
--- a/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
+++ b/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
@@ -19,8 +19,10 @@
 package org.apache.kylin.job.dao;
 
 import com.google.common.collect.Lists;
+import org.apache.commons.io.IOUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
+import org.apache.kylin.common.persistence.RawResource;
 import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.persistence.Serializer;
 import org.apache.kylin.job.exception.PersistentException;
@@ -28,6 +30,7 @@ import org.apache.kylin.metadata.MetadataManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.DataInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -35,7 +38,6 @@ import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
 /**
- * Created by qianzhou on 12/15/14.
  */
 public class ExecutableDao {
 
@@ -99,11 +101,10 @@ public class ExecutableDao {
             if (resources == null) {
                 return Collections.emptyList();
             }
-            ArrayList<ExecutableOutputPO> result = new ArrayList<ExecutableOutputPO>(resources.size());
-            for (String path : resources) {
-                result.add(readJobOutputResource(path));
-            }
-            return result;
+            Collections.sort(resources);
+            String rangeStart = resources.get(0);
+            String rangeEnd = resources.get(resources.size() - 1);
+            return store.getAllResources(rangeStart, rangeEnd, ExecutableOutputPO.class, JOB_OUTPUT_SERIALIZER);
         } catch (IOException e) {
             logger.error("error get all Jobs:", e);
             throw new PersistentException(e);
@@ -112,15 +113,14 @@ public class ExecutableDao {
 
     public List<ExecutablePO> getJobs() throws PersistentException {
         try {
-            ArrayList<String> resources = store.listResources(JOB_PATH_ROOT);
-            if (resources == null) {
+            final List<String> jobIds = store.listResources(JOB_PATH_ROOT);
+            if (jobIds== null || jobIds.isEmpty()) {
                 return Collections.emptyList();
             }
-            ArrayList<ExecutablePO> result = new ArrayList<ExecutablePO>(resources.size());
-            for (String path : resources) {
-                result.add(readJobResource(path));
-            }
-            return result;
+            Collections.sort(jobIds);
+            String rangeStart = jobIds.get(0);
+            String rangeEnd = jobIds.get(jobIds.size() - 1);
+            return store.getAllResources(rangeStart, rangeEnd, ExecutablePO.class, JOB_SERIALIZER);
         } catch (IOException e) {
             logger.error("error get all Jobs:", e);
             throw new PersistentException(e);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java b/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
index ee1607a..d5b801b 100644
--- a/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ b/job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -22,14 +22,14 @@ import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.LogTitlePrinter;
 import org.apache.kylin.common.util.MailService;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.impl.threadpool.DefaultContext;
 import org.apache.kylin.job.manager.ExecutableManager;
-import org.apache.commons.lang.StringUtils;
-import org.apache.commons.lang3.tuple.Pair;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -41,7 +41,6 @@ import java.util.Map;
 import java.util.UUID;
 
 /**
- * Created by qianzhou on 12/16/14.
  */
 public abstract class AbstractExecutable implements Executable, Idempotent {
 
@@ -232,7 +231,30 @@ public abstract class AbstractExecutable implements Executable, Idempotent {
     }
 
     protected long getExtraInfoAsLong(String key, long defaultValue) {
-        final String str = executableManager.getOutput(getId()).getExtra().get(key);
+        return getExtraInfoAsLong(executableManager.getOutput(getId()), key, defaultValue);
+    }
+
+    public static long getStartTime(Output output) {
+        return getExtraInfoAsLong(output, START_TIME, 0L);
+    }
+
+    public static long getEndTime(Output output) {
+        return getExtraInfoAsLong(output, END_TIME, 0L);
+    }
+
+    public static long getDuration(long startTime, long endTime) {
+        if (startTime == 0) {
+            return 0;
+        }
+        if (endTime == 0) {
+            return System.currentTimeMillis() - startTime;
+        } else {
+            return endTime - startTime;
+        }
+    }
+
+    public static long getExtraInfoAsLong(Output output, String key, long defaultValue) {
+        final String str = output.getExtra().get(key);
         if (str != null) {
             return Long.parseLong(str);
         } else {
@@ -261,16 +283,7 @@ public abstract class AbstractExecutable implements Executable, Idempotent {
     }
 
     public final long getDuration() {
-        final long startTime = getStartTime();
-        if (startTime == 0) {
-            return 0;
-        }
-        final long endTime = getEndTime();
-        if (endTime == 0) {
-            return System.currentTimeMillis() - startTime;
-        } else {
-            return endTime - startTime;
-        }
+        return getDuration(getStartTime(), getEndTime());
     }
 
     /*

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java b/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
index f51ff69..a2b310b 100644
--- a/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
+++ b/job/src/main/java/org/apache/kylin/job/manager/ExecutableManager.java
@@ -18,13 +18,10 @@
 
 package org.apache.kylin.job.manager;
 
-import java.lang.reflect.Constructor;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import javax.annotation.Nullable;
-
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.job.dao.ExecutableDao;
@@ -32,21 +29,18 @@ import org.apache.kylin.job.dao.ExecutableOutputPO;
 import org.apache.kylin.job.dao.ExecutablePO;
 import org.apache.kylin.job.exception.IllegalStateTranferException;
 import org.apache.kylin.job.exception.PersistentException;
-import org.apache.kylin.job.execution.AbstractExecutable;
-import org.apache.kylin.job.execution.DefaultChainedExecutable;
-import org.apache.kylin.job.execution.DefaultOutput;
-import org.apache.kylin.job.execution.ExecutableState;
-import org.apache.kylin.job.execution.Output;
+import org.apache.kylin.job.execution.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Function;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import javax.annotation.Nullable;
+import java.lang.reflect.Constructor;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 /**
- * Created by qianzhou on 12/16/14.
  */
 public class ExecutableManager {
 
@@ -120,18 +114,36 @@ public class ExecutableManager {
         try {
             final ExecutableOutputPO jobOutput = executableDao.getJobOutput(uuid);
             Preconditions.checkArgument(jobOutput != null, "there is no related output for job id:" + uuid);
-            final DefaultOutput result = new DefaultOutput();
-            result.setExtra(jobOutput.getInfo());
-            result.setState(ExecutableState.valueOf(jobOutput.getStatus()));
-            result.setVerboseMsg(jobOutput.getContent());
-            result.setLastModified(jobOutput.getLastModified());
-            return result;
+            return parseOutput(jobOutput);
         } catch (PersistentException e) {
             logger.error("fail to get job output:" + uuid, e);
             throw new RuntimeException(e);
         }
     }
 
+    private DefaultOutput parseOutput(ExecutableOutputPO jobOutput) {
+        final DefaultOutput result = new DefaultOutput();
+        result.setExtra(jobOutput.getInfo());
+        result.setState(ExecutableState.valueOf(jobOutput.getStatus()));
+        result.setVerboseMsg(jobOutput.getContent());
+        result.setLastModified(jobOutput.getLastModified());
+        return result;
+    }
+
+    public Map<String, Output> getAllOutputs() {
+        try {
+            final List<ExecutableOutputPO> jobOutputs = executableDao.getJobOutputs();
+            HashMap<String, Output> result = Maps.newHashMap();
+            for (ExecutableOutputPO jobOutput : jobOutputs) {
+                result.put(jobOutput.getId(), parseOutput(jobOutput));
+            }
+            return result;
+        } catch (PersistentException e) {
+            logger.error("fail to get all job output:", e);
+            throw new RuntimeException(e);
+        }
+    }
+
     public List<AbstractExecutable> getAllExecutables() {
         try {
             return Lists.transform(executableDao.getJobs(), new Function<ExecutablePO, AbstractExecutable>() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/server/src/main/java/org/apache/kylin/rest/service/BasicService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/BasicService.java b/server/src/main/java/org/apache/kylin/rest/service/BasicService.java
index c0f366d..6043edb 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/BasicService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/BasicService.java
@@ -32,6 +32,7 @@ import org.apache.kylin.invertedindex.IIManager;
 import org.apache.kylin.job.cube.CubingJob;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.job.execution.Output;
 import org.apache.kylin.job.manager.ExecutableManager;
 import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.project.ProjectInstance;
@@ -57,10 +58,7 @@ import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
@@ -176,7 +174,7 @@ public abstract class BasicService {
         return IIManager.getInstance(getConfig());
     }
 
-    protected List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList) {
+    protected List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList, final Map<String, Output> allOutputs) {
         List<CubingJob> results = Lists.newArrayList(FluentIterable.from(getExecutableManager().getAllExecutables()).filter(new Predicate<AbstractExecutable>() {
             @Override
             public boolean apply(AbstractExecutable executable) {
@@ -184,7 +182,7 @@ public abstract class BasicService {
                     if (cubeName == null) {
                         return true;
                     }
-                    return ((CubingJob) executable).getCubeName().equalsIgnoreCase(cubeName);                    
+                    return ((CubingJob) executable).getCubeName().equalsIgnoreCase(cubeName);
                 } else {
                     return false;
                 }
@@ -207,14 +205,18 @@ public abstract class BasicService {
         }).filter(new Predicate<CubingJob>() {
             @Override
             public boolean apply(CubingJob executable) {
-                return statusList.contains(executable.getStatus());
+                return statusList.contains(allOutputs.get(executable.getId()).getState());
             }
         }));
         return results;
     }
 
+    protected List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName, final Set<ExecutableState> statusList) {
+        return listAllCubingJobs(cubeName, projectName, statusList, getExecutableManager().getAllOutputs());
+    }
+
     protected List<CubingJob> listAllCubingJobs(final String cubeName, final String projectName) {
-        return listAllCubingJobs(cubeName, projectName, EnumSet.allOf(ExecutableState.class));
+        return listAllCubingJobs(cubeName, projectName, EnumSet.allOf(ExecutableState.class), getExecutableManager().getAllOutputs());
     }
 
     protected static void close(ResultSet resultSet, Statement stat, Connection conn) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/29bb731d/server/src/main/java/org/apache/kylin/rest/service/JobService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/JobService.java b/server/src/main/java/org/apache/kylin/rest/service/JobService.java
index 7870fdc..b3aa4ee 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/JobService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/JobService.java
@@ -57,7 +57,7 @@ import java.util.*;
 public class JobService extends BasicService {
 
     @SuppressWarnings("unused")
-    private static final Logger logger = LoggerFactory.getLogger(CubeService.class);
+    private static final Logger logger = LoggerFactory.getLogger(JobService.class);
 
     public List<JobInstance> listAllJobs(final String cubeName, final String projectName, final List<JobStatusEnum> statusList, final Integer limitValue, final Integer offsetValue) throws IOException, JobException {
         Integer limit = (null == limitValue) ? 30 : limitValue;
@@ -90,10 +90,11 @@ public class JobService extends BasicService {
                 states.add(parseToExecutableState(status));
             }
         }
-        return Lists.newArrayList(FluentIterable.from(listAllCubingJobs(cubeName, projectName, states)).transform(new Function<CubingJob, JobInstance>() {
+        final Map<String, Output> allOutputs = getExecutableManager().getAllOutputs();
+        return Lists.newArrayList(FluentIterable.from(listAllCubingJobs(cubeName, projectName, states, allOutputs)).transform(new Function<CubingJob, JobInstance>() {
             @Override
             public JobInstance apply(CubingJob cubingJob) {
-                return parseToJobInstance(cubingJob);
+                return parseToJobInstance(cubingJob, allOutputs);
             }
         }));
     }
@@ -149,18 +150,18 @@ public class JobService extends BasicService {
             throw new JobException("invalid build type:" + buildType);
         }
         getExecutableManager().addJob(job);
-        return parseToJobInstance(job);
+        return getSingleJobInstance(job);
     }
 
     public JobInstance getJobInstance(String uuid) throws IOException, JobException {
-        return parseToJobInstance(getExecutableManager().getJob(uuid));
+        return getSingleJobInstance(getExecutableManager().getJob(uuid));
     }
 
     public Output getOutput(String id) {
         return getExecutableManager().getOutput(id);
     }
 
-    private JobInstance parseToJobInstance(AbstractExecutable job) {
+    private JobInstance getSingleJobInstance(AbstractExecutable job) {
         if (job == null) {
             return null;
         }
@@ -179,31 +180,56 @@ public class JobService extends BasicService {
         result.setDuration(cubeJob.getDuration() / 1000);
         for (int i = 0; i < cubeJob.getTasks().size(); ++i) {
             AbstractExecutable task = cubeJob.getTasks().get(i);
-            result.addStep(parseToJobStep(task, i));
+            result.addStep(parseToJobStep(task, i, getExecutableManager().getOutput(task.getId())));
         }
         return result;
     }
 
-    private JobInstance.JobStep parseToJobStep(AbstractExecutable task, int i) {
+    private JobInstance parseToJobInstance(AbstractExecutable job, Map<String, Output> outputs) {
+        if (job == null) {
+            return null;
+        }
+        Preconditions.checkState(job instanceof CubingJob, "illegal job type, id:" + job.getId());
+        CubingJob cubeJob = (CubingJob) job;
+        Output output = outputs.get(job.getId());
+        final JobInstance result = new JobInstance();
+        result.setName(job.getName());
+        result.setRelatedCube(cubeJob.getCubeName());
+        result.setRelatedSegment(cubeJob.getSegmentId());
+        result.setLastModified(output.getLastModified());
+        result.setSubmitter(cubeJob.getSubmitter());
+        result.setUuid(cubeJob.getId());
+        result.setType(CubeBuildTypeEnum.BUILD);
+        result.setStatus(parseToJobStatus(output.getState()));
+        result.setMrWaiting(AbstractExecutable.getExtraInfoAsLong(output, CubingJob.MAP_REDUCE_WAIT_TIME, 0L) / 1000);
+        result.setDuration(AbstractExecutable.getDuration(AbstractExecutable.getStartTime(output), AbstractExecutable.getEndTime(output)) / 1000);
+        for (int i = 0; i < cubeJob.getTasks().size(); ++i) {
+            AbstractExecutable task = cubeJob.getTasks().get(i);
+            result.addStep(parseToJobStep(task, i, outputs.get(task.getId())));
+        }
+        return result;
+    }
+
+    private JobInstance.JobStep parseToJobStep(AbstractExecutable task, int i, Output stepOutput) {
+        Preconditions.checkNotNull(stepOutput);
         JobInstance.JobStep result = new JobInstance.JobStep();
         result.setId(task.getId());
         result.setName(task.getName());
         result.setSequenceID(i);
-        result.setStatus(parseToJobStepStatus(task.getStatus()));
-        final Output output = getExecutableManager().getOutput(task.getId());
-        for (Map.Entry<String, String> entry : output.getExtra().entrySet()) {
+        result.setStatus(parseToJobStepStatus(stepOutput.getState()));
+        for (Map.Entry<String, String> entry : stepOutput.getExtra().entrySet()) {
             if (entry.getKey() != null && entry.getValue() != null) {
                 result.putInfo(entry.getKey(), entry.getValue());
             }
         }
-        result.setExecStartTime(task.getStartTime());
-        result.setExecEndTime(task.getEndTime());
+        result.setExecStartTime(AbstractExecutable.getStartTime(stepOutput));
+        result.setExecEndTime(AbstractExecutable.getEndTime(stepOutput));
         if (task instanceof ShellExecutable) {
             result.setExecCmd(((ShellExecutable) task).getCmd());
         }
         if (task instanceof MapReduceExecutable) {
             result.setExecCmd(((MapReduceExecutable) task).getMapReduceParams());
-            result.setExecWaitTime(((MapReduceExecutable) task).getMapReduceWaitTime() / 1000);
+            result.setExecWaitTime(AbstractExecutable.getExtraInfoAsLong(stepOutput, MapReduceExecutable.MAP_REDUCE_WAIT_TIME, 0L) / 1000);
         }
         if (task instanceof HadoopShellExecutable) {
             result.setExecCmd(((HadoopShellExecutable) task).getJobParams());


[06/13] incubator-kylin git commit: KYLIN-780, query compile pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java b/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
index 5198fd7..cee4de4 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/Driver.java
@@ -21,12 +21,12 @@ package org.apache.kylin.jdbc;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.DriverVersion;
-import net.hydromatic.avatica.Handler;
-import net.hydromatic.avatica.HandlerImpl;
-import net.hydromatic.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.DriverVersion;
+import org.apache.calcite.avatica.Handler;
+import org.apache.calcite.avatica.HandlerImpl;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 import org.apache.kylin.jdbc.stub.RemoteClient;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnectionImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnectionImpl.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnectionImpl.java
index 887bd66..718ef7d 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnectionImpl.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinConnectionImpl.java
@@ -27,12 +27,12 @@ import java.util.Properties;
 
 import javax.xml.bind.DatatypeConverter;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaFactory;
-import net.hydromatic.avatica.AvaticaPreparedStatement;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.Meta;
-import net.hydromatic.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinEnumerator.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinEnumerator.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinEnumerator.java
index 04d9231..2502d4b 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinEnumerator.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinEnumerator.java
@@ -21,7 +21,7 @@ package org.apache.kylin.jdbc;
 import java.util.Collection;
 import java.util.Iterator;
 
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.linq4j.Enumerator;
 
 /**
  * Query results enumerator

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbc41Factory.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbc41Factory.java
index 90ff2e2..1336f96 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbc41Factory.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinJdbc41Factory.java
@@ -24,16 +24,16 @@ import java.util.List;
 import java.util.Properties;
 import java.util.TimeZone;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaDatabaseMetaData;
-import net.hydromatic.avatica.AvaticaFactory;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaPreparedStatement;
-import net.hydromatic.avatica.AvaticaResultSet;
-import net.hydromatic.avatica.AvaticaResultSetMetaData;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.UnregisteredDriver;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaDatabaseMetaData;
+import org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaResultSetMetaData;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.UnregisteredDriver;
 
 import org.apache.kylin.jdbc.stub.KylinClient;
 import org.apache.kylin.jdbc.stub.RemoteClient;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMetaImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMetaImpl.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMetaImpl.java
index 3c14c9b..17e8ed4 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMetaImpl.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinMetaImpl.java
@@ -29,25 +29,19 @@ import java.util.Collections;
 import java.util.List;
 import java.util.regex.Pattern;
 
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaResultSet;
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.ColumnMetaData.Rep;
-import net.hydromatic.avatica.Cursor;
-import net.hydromatic.avatica.Meta;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.optiq.runtime.EnumeratorCursor;
-
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.sql.SqlJdbcFunctionCall;
+import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.kylin.jdbc.stub.DataSet;
 import org.apache.kylin.jdbc.stub.KylinColumnMetaData;
 import org.apache.kylin.jdbc.stub.RemoteClient;
-import org.eigenbase.sql.SqlJdbcFunctionCall;
-import org.eigenbase.sql.parser.SqlParser;
-import org.eigenbase.util.Util;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.kylin.jdbc.util.SQLTypeMap;
 
 /**
@@ -329,7 +323,7 @@ public class KylinMetaImpl implements Meta {
      * Client could request metadata after prepare
      * 
      * (non-Javadoc)
-     * @see net.hydromatic.avatica.Meta#prepare(net.hydromatic.avatica.AvaticaStatement, java.lang.String)
+     * @see org.apache.calcite.avatica.Meta#prepare(org.apache.calcite.avatica.AvaticaStatement, java.lang.String)
      */
     public AvaticaPrepareResult prepare(AvaticaStatement statement, String sql) {
         RemoteClient client = factory.newRemoteClient(conn);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
index 0e4fb35..686e948 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinPrepare.java
@@ -21,12 +21,10 @@ package org.apache.kylin.jdbc;
 import java.util.List;
 import java.util.Map;
 
-import net.hydromatic.avatica.AvaticaParameter;
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.Cursor;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.optiq.runtime.EnumeratorCursor;
+import org.apache.calcite.avatica.AvaticaParameter;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.Cursor;
 
 /**
  * Interface of kylin prepare statement implementation

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
index 28c3725..7cb9b25 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinResultSet.java
@@ -21,9 +21,9 @@ package org.apache.kylin.jdbc;
 import java.sql.ResultSetMetaData;
 import java.util.TimeZone;
 
-import net.hydromatic.avatica.AvaticaPrepareResult;
-import net.hydromatic.avatica.AvaticaResultSet;
-import net.hydromatic.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.AvaticaPrepareResult;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
 
 import org.apache.kylin.jdbc.KylinPrepare.PrepareResult;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/KylinStatementImpl.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinStatementImpl.java b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinStatementImpl.java
index b380b00..0d7605d 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/KylinStatementImpl.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/KylinStatementImpl.java
@@ -18,9 +18,9 @@
 
 package org.apache.kylin.jdbc;
 
-import net.hydromatic.avatica.AvaticaConnection;
-import net.hydromatic.avatica.AvaticaResultSet;
-import net.hydromatic.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.calcite.avatica.AvaticaStatement;
 
 /**
  * Kylin statement implementation

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/stub/DataSet.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/stub/DataSet.java b/jdbc/src/main/java/org/apache/kylin/jdbc/stub/DataSet.java
index 3584b31..8b0347a 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/stub/DataSet.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/stub/DataSet.java
@@ -20,8 +20,8 @@ package org.apache.kylin.jdbc.stub;
 
 import java.util.List;
 
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.linq4j.Enumerator;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.linq4j.Enumerator;
 
 /**
  * Data set wrapper.

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/stub/KylinClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/stub/KylinClient.java b/jdbc/src/main/java/org/apache/kylin/jdbc/stub/KylinClient.java
index ccbfe5d..67652cb 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/stub/KylinClient.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/stub/KylinClient.java
@@ -23,10 +23,10 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
 
-import net.hydromatic.avatica.AvaticaStatement;
-import net.hydromatic.avatica.ColumnMetaData;
-import net.hydromatic.avatica.ColumnMetaData.Rep;
-import net.hydromatic.avatica.ColumnMetaData.ScalarType;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.ColumnMetaData;
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
+import org.apache.calcite.avatica.ColumnMetaData.ScalarType;
 
 import org.apache.commons.httpclient.HttpClient;
 import org.apache.commons.httpclient.HttpException;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/jdbc/src/main/java/org/apache/kylin/jdbc/stub/RemoteClient.java
----------------------------------------------------------------------
diff --git a/jdbc/src/main/java/org/apache/kylin/jdbc/stub/RemoteClient.java b/jdbc/src/main/java/org/apache/kylin/jdbc/stub/RemoteClient.java
index dae4126..093f150 100644
--- a/jdbc/src/main/java/org/apache/kylin/jdbc/stub/RemoteClient.java
+++ b/jdbc/src/main/java/org/apache/kylin/jdbc/stub/RemoteClient.java
@@ -20,7 +20,7 @@ package org.apache.kylin.jdbc.stub;
 
 import java.sql.SQLException;
 
-import net.hydromatic.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.AvaticaStatement;
 
 import org.apache.kylin.jdbc.KylinMetaImpl.MetaProject;
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
index a85f47b..cc89119 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
@@ -22,14 +22,12 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.jdbc.OptiqConnection;
-
-import org.eigenbase.reltype.RelDataTypeField;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
+import org.apache.calcite.DataContext;
+import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.kylin.query.relnode.OLAPContext;
 import org.apache.kylin.storage.IStorageEngine;
 import org.apache.kylin.storage.StorageEngineFactory;
@@ -39,7 +37,6 @@ import org.apache.kylin.metadata.tuple.ITuple;
 import org.apache.kylin.metadata.tuple.ITupleIterator;
 
 /**
- * @author xjiang
  */
 public class CubeEnumerator implements Enumerator<Object[]> {
 
@@ -104,7 +101,7 @@ public class CubeEnumerator implements Enumerator<Object[]> {
             this.fieldIndexes = new int[size];
             for (int i = 0; i < size; i++) {
                 String field = fields.get(i);
-                RelDataTypeField relField = olapContext.olapRowType.getField(field, true);
+                RelDataTypeField relField = olapContext.olapRowType.getField(field, true, false);
                 if (relField != null) {
                     fieldIndexes[i] = relField.getIndex();
                 } else {
@@ -172,7 +169,7 @@ public class CubeEnumerator implements Enumerator<Object[]> {
     }
 
     private void setConnectionProperties() {
-        OptiqConnection conn = (OptiqConnection) optiqContext.getQueryProvider();
+        CalciteConnection conn = (CalciteConnection) optiqContext.getQueryProvider();
         Properties connProps = conn.getProperties();
 
         String propThreshold = connProps.getProperty(OLAPQuery.PROP_SCAN_THRESHOLD);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/enumerator/HiveEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/HiveEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/HiveEnumerator.java
index c9cec94..b9fb39c 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/HiveEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/HiveEnumerator.java
@@ -24,16 +24,12 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 
-import net.hydromatic.linq4j.Enumerator;
-
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.kylin.query.relnode.OLAPContext;
-import org.eigenbase.reltype.RelDataTypeField;
 
 /**
  * Hive Query Result Enumerator
- * 
- * @author xjiang
- * 
  */
 public class HiveEnumerator implements Enumerator<Object[]> {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
index 0a3627f..9e3e81d 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
@@ -22,8 +22,7 @@ import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 
-import net.hydromatic.linq4j.Enumerator;
-
+import org.apache.calcite.linq4j.Enumerator;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.cube.model.DimensionDesc;
@@ -34,8 +33,6 @@ import org.apache.kylin.query.schema.OLAPTable;
 import org.apache.kylin.storage.tuple.Tuple;
 
 /**
- * @author yangli9
- * 
  */
 public class LookupTableEnumerator implements Enumerator<Object[]> {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
index 090980f..453c898 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPQuery.java
@@ -18,17 +18,13 @@
 
 package org.apache.kylin.query.enumerator;
 
-import net.hydromatic.linq4j.AbstractEnumerable;
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.optiq.DataContext;
-
+import org.apache.calcite.DataContext;
+import org.apache.calcite.linq4j.AbstractEnumerable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
 import org.apache.kylin.query.relnode.OLAPContext;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPQuery extends AbstractEnumerable<Object[]> implements Enumerable<Object[]> {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
index 5c6b29b..b80a483 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPAggregateRule.java
@@ -18,35 +18,31 @@
 
 package org.apache.kylin.query.optrule;
 
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.kylin.query.relnode.OLAPRel;
-import org.eigenbase.rel.AggregateRel;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelTraitSet;
-
 import org.apache.kylin.query.relnode.OLAPAggregateRel;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPAggregateRule extends ConverterRule {
 
     public static final ConverterRule INSTANCE = new OLAPAggregateRule();
 
     public OLAPAggregateRule() {
-        super(AggregateRel.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPAggregateRule");
+        super(LogicalAggregate.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPAggregateRule");
     }
 
     @Override
     public RelNode convert(RelNode rel) {
-        AggregateRel agg = (AggregateRel) rel;
+        LogicalAggregate agg = (LogicalAggregate) rel;
         RelTraitSet traitSet = agg.getTraitSet().replace(OLAPRel.CONVENTION);
         try {
-            return new OLAPAggregateRel(agg.getCluster(), traitSet, convert(agg.getChild(), traitSet), agg.getGroupSet(), agg.getAggCallList());
+            return new OLAPAggregateRel(agg.getCluster(), traitSet, convert(agg.getInput(), traitSet), agg.getGroupSet(), agg.getAggCallList());
         } catch (InvalidRelException e) {
             throw new IllegalStateException("Can't create OLAPAggregateRel!", e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPFilterRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPFilterRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPFilterRule.java
index 619e430..4c34f4e 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPFilterRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPFilterRule.java
@@ -18,33 +18,29 @@
 
 package org.apache.kylin.query.optrule;
 
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.kylin.query.relnode.OLAPFilterRel;
 import org.apache.kylin.query.relnode.OLAPRel;
-import org.eigenbase.rel.FilterRel;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelTraitSet;
 
 /**
- * 
- * @author xjiang
- * 
  */
-
 public class OLAPFilterRule extends RelOptRule {
 
     public static final RelOptRule INSTANCE = new OLAPFilterRule();
 
     public OLAPFilterRule() {
-        super(operand(FilterRel.class, any()));
+        super(operand(LogicalFilter.class, any()));
     }
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-        FilterRel filter = call.rel(0);
+        LogicalFilter filter = call.rel(0);
 
         RelTraitSet traitSet = filter.getTraitSet().replace(OLAPRel.CONVENTION);
-        OLAPFilterRel olapFilter = new OLAPFilterRel(filter.getCluster(), traitSet, convert(filter.getChild(), traitSet), filter.getCondition());
+        OLAPFilterRel olapFilter = new OLAPFilterRel(filter.getCluster(), traitSet, convert(filter.getInput(), traitSet), filter.getCondition());
         call.transformTo(olapFilter);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
index dd97933..10047e6 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPJoinRule.java
@@ -18,35 +18,32 @@
 
 package org.apache.kylin.query.optrule;
 
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.kylin.query.relnode.OLAPFilterRel;
 import org.apache.kylin.query.relnode.OLAPJoinRel;
 import org.apache.kylin.query.relnode.OLAPRel;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.JoinInfo;
-import org.eigenbase.rel.JoinRel;
-import org.eigenbase.rel.JoinRelType;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPJoinRule extends ConverterRule {
 
     public static final ConverterRule INSTANCE = new OLAPJoinRule();
 
     public OLAPJoinRule() {
-        super(JoinRel.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPJoinRule");
+        super(LogicalJoin.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPJoinRule");
     }
 
     @Override
     public RelNode convert(RelNode rel) {
-        JoinRel join = (JoinRel) rel;
+        LogicalJoin join = (LogicalJoin) rel;
         RelNode left = join.getInput(0);
         RelNode right = join.getInput(1);
 
@@ -68,8 +65,7 @@ public class OLAPJoinRule extends ConverterRule {
                     info.getEquiCondition(left, right, cluster.getRexBuilder()), //
                     info.leftKeys, info.rightKeys, join.getJoinType(), join.getVariablesStopped());
         } catch (InvalidRelException e) {
-            // Semantic error not possible. Must be a bug. Convert to
-            // internal error.
+            // Semantic error not possible. Must be a bug. Convert to internal error.
             throw new AssertionError(e);
             // LOGGER.fine(e.toString());
             // return null;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPLimitRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPLimitRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPLimitRule.java
index 359b722..cde934e 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPLimitRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPLimitRule.java
@@ -18,35 +18,32 @@
 
 package org.apache.kylin.query.optrule;
 
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.kylin.query.relnode.OLAPLimitRel;
 import org.apache.kylin.query.relnode.OLAPRel;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SortRel;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelTraitSet;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPLimitRule extends RelOptRule {
 
     public static final RelOptRule INSTANCE = new OLAPLimitRule();
 
     public OLAPLimitRule() {
-        super(operand(SortRel.class, any()), "OLAPLimitRule");
+        super(operand(Sort.class, any()), "OLAPLimitRule");
     }
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-        final SortRel sort = call.rel(0);
+        final Sort sort = call.rel(0);
         if (sort.offset == null && sort.fetch == null) {
             return;
         }
         final RelTraitSet traitSet = sort.getTraitSet().replace(OLAPRel.CONVENTION);
-        RelNode input = sort.getChild();
+        RelNode input = sort.getInput();
         if (!sort.getCollation().getFieldCollations().isEmpty()) {
             // Create a sort with the same sort key, but no offset or fetch.
             input = sort.copy(sort.getTraitSet(), input, sort.getCollation(), null, null);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPProjectRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPProjectRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPProjectRule.java
index 7ab9dc3..4867162 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPProjectRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPProjectRule.java
@@ -18,34 +18,30 @@
 
 package org.apache.kylin.query.optrule;
 
-import org.eigenbase.rel.ProjectRel;
-import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.RelOptRuleCall;
-import org.eigenbase.relopt.RelTraitSet;
-
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.kylin.query.relnode.OLAPProjectRel;
 import org.apache.kylin.query.relnode.OLAPRel;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPProjectRule extends RelOptRule {
 
     public static final RelOptRule INSTANCE = new OLAPProjectRule();
 
     public OLAPProjectRule() {
-        super(operand(ProjectRel.class, any()));
+        super(operand(LogicalProject.class, any()));
     }
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-        ProjectRel project = call.rel(0);
+        LogicalProject project = call.rel(0);
 
         RelTraitSet traitSet = project.getTraitSet().replace(OLAPRel.CONVENTION);
         OLAPProjectRel olapProj = new OLAPProjectRel(project.getCluster(), traitSet, //
-                convert(project.getChild(), traitSet), project.getProjects(), project.getRowType(), project.getFlags());
+                convert(project.getInput(), traitSet), project.getProjects(), project.getRowType(), project.getFlags());
         call.transformTo(olapProj);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPSortRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPSortRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPSortRule.java
index c9ea5f4..c020d63 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPSortRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPSortRule.java
@@ -18,35 +18,32 @@
 
 package org.apache.kylin.query.optrule;
 
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.kylin.query.relnode.OLAPRel;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SortRel;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelTraitSet;
-
 import org.apache.kylin.query.relnode.OLAPSortRel;
 
 /**
- * @author xjiang
- * 
  */
 public class OLAPSortRule extends ConverterRule {
 
     public static final OLAPSortRule INSTANCE = new OLAPSortRule();
 
     public OLAPSortRule() {
-        super(SortRel.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPSortRule");
+        super(Sort.class, Convention.NONE, OLAPRel.CONVENTION, "OLAPSortRule");
     }
 
     @Override
     public RelNode convert(RelNode rel) {
-        final SortRel sort = (SortRel) rel;
+        final Sort sort = (Sort) rel;
         if (sort.offset != null || sort.fetch != null) {
             return null;
         }
         final RelTraitSet traitSet = sort.getTraitSet().replace(OLAPRel.CONVENTION);
-        final RelNode input = sort.getChild();
+        final RelNode input = sort.getInput();
         return new OLAPSortRel(rel.getCluster(), traitSet, convert(input, input.getTraitSet().replace(OLAPRel.CONVENTION)), sort.getCollation(), sort.offset, sort.fetch);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/optrule/OLAPToEnumerableConverterRule.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/optrule/OLAPToEnumerableConverterRule.java b/query/src/main/java/org/apache/kylin/query/optrule/OLAPToEnumerableConverterRule.java
index 4edee38..995e3e7 100644
--- a/query/src/main/java/org/apache/kylin/query/optrule/OLAPToEnumerableConverterRule.java
+++ b/query/src/main/java/org/apache/kylin/query/optrule/OLAPToEnumerableConverterRule.java
@@ -18,19 +18,14 @@
 
 package org.apache.kylin.query.optrule;
 
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.kylin.query.relnode.OLAPRel;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.convert.ConverterRule;
-import org.eigenbase.relopt.RelTraitSet;
-
 import org.apache.kylin.query.relnode.OLAPToEnumerableConverter;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPToEnumerableConverterRule extends ConverterRule {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 7cdb0b4..24d9a3e 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -19,56 +19,53 @@
 package org.apache.kylin.query.relnode;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import net.hydromatic.optiq.AggregateFunction;
-import net.hydromatic.optiq.FunctionParameter;
-import net.hydromatic.optiq.impl.AggregateFunctionImpl;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableAggregateRel;
-
+import org.apache.calcite.adapter.enumerable.EnumerableAggregate;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.schema.AggregateFunction;
+import org.apache.calcite.schema.FunctionParameter;
+import org.apache.calcite.schema.impl.AggregateFunctionImpl;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.InferTypes;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Util;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.query.sqlfunc.HLLDistinctCountAggFunc;
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.Aggregation;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.sql.SqlAggFunction;
-import org.eigenbase.sql.SqlIdentifier;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.type.InferTypes;
-import org.eigenbase.sql.type.OperandTypes;
-import org.eigenbase.sql.type.ReturnTypes;
-import org.eigenbase.sql.type.SqlTypeFamily;
-import org.eigenbase.sql.validate.SqlUserDefinedAggFunction;
-import org.eigenbase.util.Util;
 
 import com.google.common.base.Preconditions;
 
 /**
- * @author xjiang
  */
-public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, EnumerableRel {
+public class OLAPAggregateRel extends Aggregate implements OLAPRel, EnumerableRel {
 
     private final static Map<String, String> AGGR_FUNC_MAP = new HashMap<String, String>();
 
@@ -101,16 +98,22 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
     private List<TblColRef> groups;
     private List<FunctionDesc> aggregations;
 
-    public OLAPAggregateRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet, List<AggregateCall> aggCalls) throws InvalidRelException {
-        super(cluster, traits, child, groupSet, aggCalls);
+    public OLAPAggregateRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, ImmutableBitSet groupSet, List<AggregateCall> aggCalls) throws InvalidRelException {
+        super(cluster, traits, child, false, groupSet, asList(groupSet), aggCalls);
         Preconditions.checkArgument(getConvention() == OLAPRel.CONVENTION);
         this.afterAggregate = false;
         this.rewriteAggCalls = aggCalls;
         this.rowType = getRowType();
     }
 
+    private static List<ImmutableBitSet> asList(ImmutableBitSet groupSet) {
+        ArrayList<ImmutableBitSet> l = new ArrayList<ImmutableBitSet>(1);
+        l.add(groupSet);
+        return l;
+    }
+
     @Override
-    public AggregateRelBase copy(RelTraitSet traitSet, RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
+    public Aggregate copy(RelTraitSet traitSet, RelNode input, boolean indicator, ImmutableBitSet groupSet, List<ImmutableBitSet> groupSets, List<AggregateCall> aggCalls) {
         try {
             return new OLAPAggregateRel(getCluster(), traitSet, input, groupSet, aggCalls);
         } catch (InvalidRelException e) {
@@ -120,19 +123,20 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
 
     @Override
     public RelOptCost computeSelfCost(RelOptPlanner planner) {
-        double factor = .5;
-        for (AggregateCall aggCall : aggCalls) {
-            if ("$SUM0".equals(aggCall.getAggregation().getName())) {
-                factor = .2;
-            }
-        }
-        return super.computeSelfCost(planner).multiplyBy(factor);
+        return super.computeSelfCost(planner).multiplyBy(.05);
+//        double factor = .5;
+//        for (AggregateCall aggCall : aggCalls) {
+//            if ("$SUM0".equals(aggCall.getAggregation().getName())) {
+//                factor = .2;
+//            }
+//        }
+//        return super.computeSelfCost(planner).multiplyBy(factor);
     }
 
     @Override
     public void implementOLAP(OLAPImplementor implementor) {
 
-        implementor.visitChild(getChild(), this);
+        implementor.visitChild(getInput(), this);
 
         this.context = implementor.getContext();
         this.columnRowType = buildColumnRowType();
@@ -158,7 +162,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
         buildGroups();
         buildAggregations();
 
-        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
         List<TblColRef> columns = new ArrayList<TblColRef>(this.rowType.getFieldCount());
         columns.addAll(this.groups);
 
@@ -192,7 +196,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
     }
 
     private void buildGroups() {
-        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
         this.groups = new ArrayList<TblColRef>();
         for (int i = getGroupSet().nextSetBit(0); i >= 0; i = getGroupSet().nextSetBit(i + 1)) {
             Set<TblColRef> columns = inputColumnRowType.getSourceColumnsByIndex(i);
@@ -201,7 +205,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
     }
 
     private void buildAggregations() {
-        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
         this.aggregations = new ArrayList<FunctionDesc>();
         for (AggregateCall aggCall : this.rewriteAggCalls) {
             ParameterDesc parameter = null;
@@ -227,7 +231,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
     }
 
     private void translateAggregation() {
-        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
         for (int i = 0; i < this.aggregations.size(); i++) {
             FunctionDesc aggFunc = this.aggregations.get(i);
             context.aggregations.add(aggFunc);
@@ -251,14 +255,13 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
     }
 
     private void fillbackOptimizedColumn() {
-        // some aggcall will be optimized out in sub-query (e.g. tableau generated sql)
-        // we need to fill them back
-        RelDataType inputAggRow = getChild().getRowType();
+        // some aggcall will be optimized out in sub-query (e.g. tableau generated sql), we need to fill them back
+        RelDataType inputAggRow = getInput().getRowType();
         RelDataType outputAggRow = getRowType();
         if (inputAggRow.getFieldCount() != outputAggRow.getFieldCount()) {
             for (RelDataTypeField inputField : inputAggRow.getFieldList()) {
                 String inputFieldName = inputField.getName();
-                if (outputAggRow.getField(inputFieldName, true) == null) {
+                if (outputAggRow.getField(inputFieldName, true, false) == null) {
                     TblColRef column = this.columnRowType.getColumnByIndex(inputField.getIndex());
                     this.context.metricsColumns.add(column);
                 }
@@ -268,7 +271,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
 
     @Override
     public void implementRewrite(RewriteImplementor implementor) {
-        implementor.visitChild(this, getChild());
+        implementor.visitChild(this, getInput());
 
         // only rewrite the first aggregation
         if (!this.afterAggregate && RewriteImplementor.needRewrite(this.context)) {
@@ -295,12 +298,12 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
         // rebuild parameters
         List<Integer> newArgList = new ArrayList<Integer>(1);
         String fieldName = func.getRewriteFieldName();
-        RelDataTypeField field = getChild().getRowType().getField(fieldName, true);
+        RelDataTypeField field = getInput().getRowType().getField(fieldName, true, false);
         newArgList.add(field.getIndex());
 
         // rebuild function
         RelDataType fieldType = aggCall.getType();
-        Aggregation newAgg = aggCall.getAggregation();
+        SqlAggFunction newAgg = aggCall.getAggregation();
         if (func.isCountDistinct()) {
             newAgg = createHyperLogLogAggFunction(fieldType);
         } else if (func.isCount()) {
@@ -322,7 +325,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
         return new AggregateCall(newAgg, false, newArgList, inferReturnType, newAgg.getName());
     }
 
-    private Aggregation createHyperLogLogAggFunction(RelDataType returnType) {
+    private SqlAggFunction createHyperLogLogAggFunction(RelDataType returnType) {
         RelDataTypeFactory typeFactory = getCluster().getTypeFactory();
         SqlIdentifier sqlIdentifier = new SqlIdentifier("HLL_COUNT", new SqlParserPos(1, 1));
         AggregateFunction aggFunction = AggregateFunctionImpl.create(HLLDistinctCountAggFunc.class);
@@ -339,9 +342,9 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
     @Override
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
 
-        EnumerableAggregateRel enumAggRel;
+        EnumerableAggregate enumAggRel;
         try {
-            enumAggRel = new EnumerableAggregateRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getChild(), this.groupSet, rewriteAggCalls);
+            enumAggRel = new EnumerableAggregate(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getInput(), false, this.groupSet, this.groupSets, rewriteAggCalls);
         } catch (InvalidRelException e) {
             throw new IllegalStateException("Can't create EnumerableAggregateRel!", e);
         }
@@ -361,7 +364,7 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
 
     @Override
     public boolean hasSubQuery() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         return olapChild.hasSubQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
index 8e532a9..86473b5 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPContext.java
@@ -26,10 +26,9 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.kylin.metadata.realization.SQLDigest;
 import org.apache.kylin.query.schema.OLAPSchema;
-import org.eigenbase.reltype.RelDataType;
-
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.TblColRef;
@@ -38,8 +37,6 @@ import org.apache.kylin.storage.StorageContext;
 import org.apache.kylin.metadata.filter.TupleFilter;
 
 /**
- * @author xjiang
- * 
  */
 public class OLAPContext {
 
@@ -95,8 +92,7 @@ public class OLAPContext {
 
     // query info
     public OLAPSchema olapSchema = null;
-    public OLAPTableScan firstTableScan = null; // to be fact table scan except
-                                                // "select * from lookupTable"
+    public OLAPTableScan firstTableScan = null; // to be fact table scan except "select * from lookupTable"
     public RelDataType olapRowType = null;
     public boolean afterAggregate = false;
     public boolean afterJoin = false;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index 0783e31..7d16c8d 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -18,41 +18,39 @@
 
 package org.apache.kylin.query.relnode;
 
-import java.util.*;
-
-import com.google.common.collect.Sets;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableCalcRel;
-import net.hydromatic.optiq.runtime.SqlFunctions;
-
-import org.eigenbase.rel.FilterRelBase;
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexBuilder;
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.rex.RexDynamicParam;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexLocalRef;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexProgram;
-import org.eigenbase.rex.RexProgramBuilder;
-import org.eigenbase.rex.RexVisitorImpl;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.SqlOperator;
-import org.eigenbase.util.NlsString;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableList;
-import org.apache.kylin.metadata.model.TblColRef;
+import java.util.Calendar;
+import java.util.GregorianCalendar;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.calcite.adapter.enumerable.EnumerableCalc;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.NlsString;
 import org.apache.kylin.metadata.filter.CaseTupleFilter;
 import org.apache.kylin.metadata.filter.ColumnTupleFilter;
 import org.apache.kylin.metadata.filter.CompareTupleFilter;
@@ -62,11 +60,15 @@ import org.apache.kylin.metadata.filter.ExtractTupleFilter;
 import org.apache.kylin.metadata.filter.LogicalTupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter;
 import org.apache.kylin.metadata.filter.TupleFilter.FilterOperatorEnum;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
 
 /**
- * @author xjiang
  */
-public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableRel {
+public class OLAPFilterRel extends Filter implements OLAPRel, EnumerableRel {
 
     private static class TupleFilterVisitor extends RexVisitorImpl<TupleFilter> {
 
@@ -211,9 +213,9 @@ public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableR
             } else if (literalValue instanceof GregorianCalendar) {
                 GregorianCalendar g = (GregorianCalendar) literalValue;
                 strValue = "" + g.get(Calendar.YEAR) + "-" + normToTwoDigits(g.get(Calendar.MONTH) + 1) + "-" + normToTwoDigits(g.get(Calendar.DAY_OF_MONTH));
-            } else if (literalValue instanceof SqlFunctions.TimeUnitRange) {
+            } else if (literalValue instanceof TimeUnitRange) {
                 // Extract(x from y) in where clause
-                strValue = ((SqlFunctions.TimeUnitRange) literalValue).name();
+                strValue = ((TimeUnitRange) literalValue).name();
             } else if (literalValue == null) {
                 strValue = null;
             } else {
@@ -247,13 +249,13 @@ public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableR
     }
 
     @Override
-    public FilterRelBase copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+    public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
         return new OLAPFilterRel(getCluster(), traitSet, input, condition);
     }
 
     @Override
     public void implementOLAP(OLAPImplementor implementor) {
-        implementor.visitChild(getChild(), this);
+        implementor.visitChild(getInput(), this);
 
         this.columnRowType = buildColumnRowType();
         this.context = implementor.getContext();
@@ -265,7 +267,7 @@ public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableR
     }
 
     private ColumnRowType buildColumnRowType() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
         return inputColumnRowType;
     }
@@ -299,20 +301,20 @@ public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableR
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
         // keep it for having clause
         RexBuilder rexBuilder = getCluster().getRexBuilder();
-        RelDataType inputRowType = getChild().getRowType();
+        RelDataType inputRowType = getInput().getRowType();
         RexProgramBuilder programBuilder = new RexProgramBuilder(inputRowType, rexBuilder);
         programBuilder.addIdentity();
         programBuilder.addCondition(this.condition);
         RexProgram program = programBuilder.getProgram();
 
-        EnumerableCalcRel enumCalcRel = new EnumerableCalcRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getChild(), this.rowType, program, ImmutableList.<RelCollation> of());
+        EnumerableCalc enumCalcRel = new EnumerableCalc(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getInput(), program, ImmutableList.<RelCollation> of());
 
         return enumCalcRel.implement(implementor, pref);
     }
 
     @Override
     public void implementRewrite(RewriteImplementor implementor) {
-        implementor.visitChild(this, getChild());
+        implementor.visitChild(this, getInput());
 
         this.rowType = this.deriveRowType();
         this.columnRowType = buildColumnRowType();
@@ -330,7 +332,7 @@ public class OLAPFilterRel extends FilterRelBase implements OLAPRel, EnumerableR
 
     @Override
     public boolean hasSubQuery() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         return olapChild.hasSubQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index c51196a..cb0c65f 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -25,42 +25,40 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import net.hydromatic.linq4j.expressions.Blocks;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableJoinRel;
-import net.hydromatic.optiq.rules.java.PhysType;
-import net.hydromatic.optiq.rules.java.PhysTypeImpl;
-
-import org.apache.kylin.query.schema.OLAPTable;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.JoinInfo;
-import org.eigenbase.rel.JoinRelType;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory.FieldInfoBuilder;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelDataTypeFieldImpl;
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.util.ImmutableIntList;
-
 import com.google.common.base.Preconditions;
+
+import org.apache.calcite.adapter.enumerable.EnumerableJoin;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableIntList;
 import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.TblColRef;
+import org.apache.kylin.query.schema.OLAPTable;
 
 /**
- * @author xjiang
  */
-public class OLAPJoinRel extends EnumerableJoinRel implements OLAPRel {
+public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
 
     private final static String[] COLUMN_ARRAY_MARKER = new String[0];
 
@@ -79,7 +77,7 @@ public class OLAPJoinRel extends EnumerableJoinRel implements OLAPRel {
     }
 
     @Override
-    public EnumerableJoinRel copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, //
+    public EnumerableJoin copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, //
             JoinRelType joinType, boolean semiJoinDone) {
 
         final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
@@ -87,8 +85,7 @@ public class OLAPJoinRel extends EnumerableJoinRel implements OLAPRel {
         try {
             return new OLAPJoinRel(getCluster(), traitSet, left, right, condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType, variablesStopped);
         } catch (InvalidRelException e) {
-            // Semantic error not possible. Must be a bug. Convert to
-            // internal error.
+            // Semantic error not possible. Must be a bug. Convert to internal error.
             throw new AssertionError(e);
         }
     }
@@ -259,7 +256,7 @@ public class OLAPJoinRel extends EnumerableJoinRel implements OLAPRel {
             List<RelDataTypeField> newFieldList = new LinkedList<RelDataTypeField>();
             for (Map.Entry<String, RelDataType> rewriteField : this.context.rewriteFields.entrySet()) {
                 String fieldName = rewriteField.getKey();
-                if (this.rowType.getField(fieldName, true) == null) {
+                if (this.rowType.getField(fieldName, true, false) == null) {
                     RelDataType fieldType = rewriteField.getValue();
                     RelDataTypeField newField = new RelDataTypeFieldImpl(fieldName, paramIndex++, fieldType);
                     newFieldList.add(newField);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
index e638fe7..ac9f3b1 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
@@ -20,28 +20,24 @@ package org.apache.kylin.query.relnode;
 
 import java.util.List;
 
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableLimitRel;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.RelWriter;
-import org.eigenbase.rel.SingleRel;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableLimit;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
 
 import com.google.common.base.Preconditions;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
 
@@ -75,7 +71,7 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
 
     @Override
     public void implementOLAP(OLAPImplementor implementor) {
-        implementor.visitChild(getChild(), this);
+        implementor.visitChild(getInput(), this);
 
         this.columnRowType = buildColumnRowType();
 
@@ -86,14 +82,14 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
     }
 
     private ColumnRowType buildColumnRowType() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
         return inputColumnRowType;
     }
 
     @Override
     public void implementRewrite(RewriteImplementor implementor) {
-        implementor.visitChild(this, getChild());
+        implementor.visitChild(this, getInput());
 
         this.rowType = this.deriveRowType();
         this.columnRowType = buildColumnRowType();
@@ -101,10 +97,10 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
 
     @Override
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-        OLAPRel childRel = (OLAPRel) getChild();
+        OLAPRel childRel = (OLAPRel) getInput();
         childRel.replaceTraitSet(EnumerableConvention.INSTANCE);
 
-        EnumerableLimitRel enumLimit = new EnumerableLimitRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getChild(), localOffset, localFetch);
+        EnumerableLimit enumLimit = new EnumerableLimit(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getInput(), localOffset, localFetch);
         Result res = enumLimit.implement(implementor, pref);
 
         childRel.replaceTraitSet(CONVENTION);
@@ -124,7 +120,7 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
 
     @Override
     public boolean hasSubQuery() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         return olapChild.hasSubQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
index 1215f9e..2de43e3 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
@@ -25,44 +25,40 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableCalcRel;
-
+import org.apache.calcite.adapter.enumerable.EnumerableCalc;
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory.FieldInfoBuilder;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlCaseOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.TblColRef.InnerDataTypeEnum;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory.FieldInfoBuilder;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelDataTypeFieldImpl;
-import org.eigenbase.rex.RexCall;
-import org.eigenbase.rex.RexInputRef;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.rex.RexProgram;
-import org.eigenbase.sql.SqlOperator;
-import org.eigenbase.sql.fun.SqlCaseOperator;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
-import org.eigenbase.sql.validate.SqlUserDefinedFunction;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 /**
- * 
- * @author xjiang
- * 
  */
-public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, EnumerableRel {
+public class OLAPProjectRel extends Project implements OLAPRel, EnumerableRel {
 
     private OLAPContext context;
     private List<RexNode> rewriteProjects;
@@ -98,13 +94,13 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
     }
 
     @Override
-    public ProjectRelBase copy(RelTraitSet traitSet, RelNode child, List<RexNode> exps, RelDataType rowType) {
+    public Project copy(RelTraitSet traitSet, RelNode child, List<RexNode> exps, RelDataType rowType) {
         return new OLAPProjectRel(getCluster(), traitSet, child, exps, rowType, this.flags);
     }
 
     @Override
     public void implementOLAP(OLAPImplementor implementor) {
-        implementor.visitChild(getChild(), this);
+        implementor.visitChild(getInput(), this);
 
         this.context = implementor.getContext();
         this.hasJoin = context.hasJoin;
@@ -117,7 +113,7 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
     private ColumnRowType buildColumnRowType() {
         List<TblColRef> columns = new ArrayList<TblColRef>();
         List<Set<TblColRef>> sourceColumns = new ArrayList<Set<TblColRef>>();
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
         for (int i = 0; i < this.rewriteProjects.size(); i++) {
             RexNode rex = this.rewriteProjects.get(i);
@@ -203,20 +199,20 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
 
     @Override
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-        EnumerableCalcRel enumCalcRel;
+        EnumerableCalc enumCalcRel;
 
-        RelNode child = getChild();
+        RelNode child = getInput();
         if (child instanceof OLAPFilterRel) {
             // merge project & filter
-            OLAPFilterRel filter = (OLAPFilterRel) getChild();
-            RexProgram program = RexProgram.create(filter.getChild().getRowType(), this.rewriteProjects, filter.getCondition(), this.rowType, getCluster().getRexBuilder());
+            OLAPFilterRel filter = (OLAPFilterRel) getInput();
+            RexProgram program = RexProgram.create(filter.getInput().getRowType(), this.rewriteProjects, filter.getCondition(), this.rowType, getCluster().getRexBuilder());
 
-            enumCalcRel = new EnumerableCalcRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), filter.getChild(), this.rowType, program, ImmutableList.<RelCollation> of());
+            enumCalcRel = new EnumerableCalc(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), filter.getInput(), program, ImmutableList.<RelCollation> of());
         } else {
             // keep project for tablescan
             RexProgram program = RexProgram.create(child.getRowType(), this.rewriteProjects, null, this.rowType, getCluster().getRexBuilder());
 
-            enumCalcRel = new EnumerableCalcRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), child, this.rowType, program, ImmutableList.<RelCollation> of());
+            enumCalcRel = new EnumerableCalc(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), child, program, ImmutableList.<RelCollation> of());
         }
 
         return enumCalcRel.implement(implementor, pref);
@@ -229,7 +225,7 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
 
     @Override
     public void implementRewrite(RewriteImplementor implementor) {
-        implementor.visitChild(this, getChild());
+        implementor.visitChild(this, getInput());
 
         this.rewriting = true;
 
@@ -243,7 +239,7 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
         int paramIndex = this.rowType.getFieldList().size();
         List<RelDataTypeField> newFieldList = new LinkedList<RelDataTypeField>();
         List<RexNode> newExpList = new LinkedList<RexNode>();
-        ColumnRowType inputColumnRowType = ((OLAPRel) getChild()).getColumnRowType();
+        ColumnRowType inputColumnRowType = ((OLAPRel) getInput()).getColumnRowType();
 
         for (Map.Entry<String, RelDataType> rewriteField : this.context.rewriteFields.entrySet()) {
             String rewriteFieldName = rewriteField.getKey();
@@ -256,7 +252,7 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
                     RelDataTypeField newField = new RelDataTypeFieldImpl(rewriteFieldName, paramIndex++, fieldType);
                     newFieldList.add(newField);
                     // new project
-                    RelDataTypeField inputField = getChild().getRowType().getFieldList().get(inputIndex);
+                    RelDataTypeField inputField = getInput().getRowType().getFieldList().get(inputIndex);
                     RexInputRef newFieldRef = new RexInputRef(inputField.getIndex(), inputField.getType());
                     newExpList.add(newFieldRef);
                 }
@@ -289,7 +285,7 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
 
     @Override
     public boolean hasSubQuery() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         return olapChild.hasSubQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
index 06cad92..1f6057f 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
@@ -21,20 +21,16 @@ package org.apache.kylin.query.relnode;
 import java.util.LinkedHashMap;
 import java.util.Stack;
 
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * 
- * @author xjiang
- * 
  */
 public interface OLAPRel extends RelNode {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
index 6287f68..2591719 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
@@ -18,32 +18,29 @@
 
 package org.apache.kylin.query.relnode;
 
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRules.EnumerableSortRel;
-
-import org.eigenbase.rel.RelCollation;
-import org.eigenbase.rel.RelFieldCollation;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SortRel;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.rex.RexNode;
-
 import com.google.common.base.Preconditions;
+
+import org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.EnumerableSort;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexNode;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.storage.StorageContext;
 
 /**
- * @author xjiang
- * 
  */
-public class OLAPSortRel extends SortRel implements EnumerableRel, OLAPRel {
+public class OLAPSortRel extends Sort implements EnumerableRel, OLAPRel {
 
     private ColumnRowType columnRowType;
     private OLAPContext context;
@@ -66,26 +63,26 @@ public class OLAPSortRel extends SortRel implements EnumerableRel, OLAPRel {
 
     @Override
     public void implementOLAP(OLAPImplementor implementor) {
-        implementor.visitChild(getChild(), this);
+        implementor.visitChild(getInput(), this);
 
         this.context = implementor.getContext();
         this.columnRowType = buildColumnRowType();
     }
 
     private ColumnRowType buildColumnRowType() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         ColumnRowType inputColumnRowType = olapChild.getColumnRowType();
         return inputColumnRowType;
     }
 
     @Override
     public void implementRewrite(RewriteImplementor implementor) {
-        implementor.visitChild(this, getChild());
+        implementor.visitChild(this, getInput());
 
         for (RelFieldCollation fieldCollation : this.collation.getFieldCollations()) {
             int index = fieldCollation.getFieldIndex();
             StorageContext.OrderEnum order = getOrderEnum(fieldCollation.getDirection());
-            OLAPRel olapChild = (OLAPRel) this.getChild();
+            OLAPRel olapChild = (OLAPRel) this.getInput();
             TblColRef orderCol = olapChild.getColumnRowType().getAllColumns().get(index);
             MeasureDesc measure = findMeasure(orderCol);
             if (measure != null) {
@@ -117,10 +114,10 @@ public class OLAPSortRel extends SortRel implements EnumerableRel, OLAPRel {
 
     @Override
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
-        OLAPRel childRel = (OLAPRel) getChild();
+        OLAPRel childRel = (OLAPRel) getInput();
         childRel.replaceTraitSet(EnumerableConvention.INSTANCE);
 
-        EnumerableSortRel enumSort = new EnumerableSortRel(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE, collation), getChild(), collation, offset, fetch);
+        EnumerableSort enumSort = new EnumerableSort(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE, collation), getInput(), collation, offset, fetch);
 
         Result res = enumSort.implement(implementor, pref);
 
@@ -141,7 +138,7 @@ public class OLAPSortRel extends SortRel implements EnumerableRel, OLAPRel {
 
     @Override
     public boolean hasSubQuery() {
-        OLAPRel olapChild = (OLAPRel) getChild();
+        OLAPRel olapChild = (OLAPRel) getInput();
         return olapChild.hasSubQuery();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index af76511..f149592 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -22,32 +22,36 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import net.hydromatic.linq4j.expressions.Blocks;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.PhysType;
-import net.hydromatic.optiq.rules.java.PhysTypeImpl;
-
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.AbstractConverter.ExpandConversionRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
+import org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.kylin.query.schema.OLAPSchema;
 import org.apache.kylin.query.schema.OLAPTable;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.RelWriter;
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.rel.rules.*;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelTrait;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.relopt.volcano.AbstractConverter.ExpandConversionRule;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
 
 import com.google.common.base.Preconditions;
+
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.query.optrule.OLAPAggregateRule;
@@ -59,9 +63,8 @@ import org.apache.kylin.query.optrule.OLAPSortRule;
 import org.apache.kylin.query.optrule.OLAPToEnumerableConverterRule;
 
 /**
- * @author xjiang
  */
-public class OLAPTableScan extends TableAccessRelBase implements OLAPRel, EnumerableRel {
+public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
 
     private final OLAPTable olapTable;
     private final String tableName;
@@ -115,19 +118,18 @@ public class OLAPTableScan extends TableAccessRelBase implements OLAPRel, Enumer
         planner.addRule(OLAPSortRule.INSTANCE);
 
         // since join is the entry point, we can't push filter past join
-        planner.removeRule(PushFilterPastJoinRule.FILTER_ON_JOIN);
-        planner.removeRule(PushFilterPastJoinRule.JOIN);
+        planner.removeRule(FilterJoinRule.FILTER_ON_JOIN);
+        planner.removeRule(FilterJoinRule.JOIN);
 
         // TODO : since we don't have statistic of table, the optimization of join is too cost
-        planner.removeRule(SwapJoinRule.INSTANCE);
-        planner.removeRule(PushJoinThroughJoinRule.LEFT);
-        planner.removeRule(PushJoinThroughJoinRule.RIGHT);
+        planner.removeRule(JoinCommuteRule.INSTANCE);
+        planner.removeRule(JoinPushThroughJoinRule.LEFT);
+        planner.removeRule(JoinPushThroughJoinRule.RIGHT);
 
-        // for columns in having clause will enable table scan filter rule
-        // cause kylin does not depend on MPP
-        planner.removeRule(PushFilterPastProjectRule.INSTANCE);
+        // for columns in having clause will enable table scan filter rule cause kylin does not depend on MPP
+        planner.removeRule(FilterProjectTransposeRule.INSTANCE);
         // distinct count will be split into a separated query that is joined with the left query
-        planner.removeRule(RemoveDistinctAggregateRule.INSTANCE);
+        planner.removeRule(AggregateExpandDistinctAggregatesRule.INSTANCE);
         
         // see Dec 26th email @ http://mail-archives.apache.org/mod_mbox/calcite-dev/201412.mbox/browser
         planner.removeRule(ExpandConversionRule.INSTANCE);
@@ -231,7 +233,7 @@ public class OLAPTableScan extends TableAccessRelBase implements OLAPRel, Enumer
 
         for (Map.Entry<String, RelDataType> rewriteField : rewriteFields.entrySet()) {
             String fieldName = rewriteField.getKey();
-            RelDataTypeField field = rowType.getField(fieldName, true);
+            RelDataTypeField field = rowType.getField(fieldName, true, false);
             if (field != null) {
                 RelDataType fieldType = field.getType();
                 rewriteField.setValue(fieldType);


[02/13] incubator-kylin git commit: KYLIN-838

Posted by li...@apache.org.
KYLIN-838


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/888255de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/888255de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/888255de

Branch: refs/heads/KYLIN-780
Commit: 888255ded589d830750082a8bbde9ab79dfc98bf
Parents: 29bb731
Author: qianhao.zhou <qi...@ebay.com>
Authored: Thu Jun 18 14:32:00 2015 +0800
Committer: qianhao.zhou <qi...@ebay.com>
Committed: Thu Jun 18 14:32:00 2015 +0800

----------------------------------------------------------------------
 job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/888255de/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
----------------------------------------------------------------------
diff --git a/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java b/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
index d7ee8b3..635c73e 100644
--- a/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
+++ b/job/src/main/java/org/apache/kylin/job/dao/ExecutableDao.java
@@ -98,7 +98,7 @@ public class ExecutableDao {
     public List<ExecutableOutputPO> getJobOutputs() throws PersistentException {
         try {
             ArrayList<String> resources = store.listResources(JOB_OUTPUT_ROOT);
-            if (resources == null) {
+            if (resources == null || resources.isEmpty()) {
                 return Collections.emptyList();
             }
             Collections.sort(resources);


[03/13] incubator-kylin git commit: KYLIN-780 undo the added debug info that fails regression test

Posted by li...@apache.org.
KYLIN-780 undo the added debug info that fails regression test


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/49ce3915
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/49ce3915
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/49ce3915

Branch: refs/heads/KYLIN-780
Commit: 49ce39158eb080e317925cbce6d13151c7dffd17
Parents: 888255d
Author: Li, Yang <ya...@ebay.com>
Authored: Thu Jun 18 15:12:48 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Thu Jun 18 15:12:48 2015 +0800

----------------------------------------------------------------------
 .../apache/kylin/query/relnode/OLAPAggregateRel.java  |  2 --
 .../apache/kylin/query/relnode/OLAPProjectRel.java    |  2 --
 .../query/relnode/OLAPToEnumerableConverter.java      | 14 +++++++-------
 3 files changed, 7 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/49ce3915/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 940668d..7cdb0b4 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -33,7 +33,6 @@ import net.hydromatic.optiq.rules.java.EnumerableRel;
 import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
 import net.hydromatic.optiq.rules.java.JavaRules.EnumerableAggregateRel;
 
-import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
@@ -283,7 +282,6 @@ public class OLAPAggregateRel extends AggregateRelBase implements OLAPRel, Enume
                 }
                 this.rewriteAggCalls.add(aggCall);
             }
-            ClassUtil.updateFinalField(AggregateRelBase.class, "aggCalls", this, this.rewriteAggCalls);
         }
 
         // rebuild rowType & columnRowType

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/49ce3915/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
index 9b9b07e..1215f9e 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
@@ -30,7 +30,6 @@ import net.hydromatic.optiq.rules.java.EnumerableRel;
 import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
 import net.hydromatic.optiq.rules.java.JavaRules.EnumerableCalcRel;
 
-import org.apache.kylin.common.util.ClassUtil;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.model.TblColRef.InnerDataTypeEnum;
 import org.eigenbase.rel.ProjectRelBase;
@@ -269,7 +268,6 @@ public class OLAPProjectRel extends ProjectRelBase implements OLAPRel, Enumerabl
             List<RexNode> newProjects = new ArrayList<RexNode>(this.rewriteProjects);
             newProjects.addAll(newExpList);
             this.rewriteProjects = newProjects;
-            ClassUtil.updateFinalField(ProjectRelBase.class, "exps", this, ImmutableList.copyOf(this.rewriteProjects));
 
             // rebuild row type
             FieldInfoBuilder fieldInfo = getCluster().getTypeFactory().builder();

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/49ce3915/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
index 16a4ecb..54cafe5 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
@@ -27,15 +27,19 @@ import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
 import net.hydromatic.optiq.rules.java.PhysType;
 import net.hydromatic.optiq.rules.java.PhysTypeImpl;
 
+import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.query.routing.NoRealizationFoundException;
 import org.apache.kylin.query.routing.QueryRouter;
 import org.apache.kylin.query.schema.OLAPTable;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.convert.ConverterRelImpl;
-import org.eigenbase.relopt.*;
+import org.eigenbase.relopt.ConventionTraitDef;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.sql.SqlExplainLevel;
-import org.apache.kylin.metadata.realization.IRealization;
 
 /**
  * @author xjiang
@@ -82,10 +86,6 @@ public class OLAPToEnumerableConverter extends ConverterRelImpl implements Enume
         OLAPRel.RewriteImplementor rewriteImplementor = new OLAPRel.RewriteImplementor();
         rewriteImplementor.visitChild(this, getChild());
 
-        //        String dumpPlan = RelOptUtil.dumpPlan("", this, false, SqlExplainLevel.DIGEST_ATTRIBUTES);
-        //        System.out.println("============================================================================");
-        //        System.out.println(dumpPlan);
-
         // build java implementation
         EnumerableRel child = (EnumerableRel) getChild();
         OLAPRel.JavaImplementor javaImplementor = new OLAPRel.JavaImplementor(enumImplementor);


[10/13] incubator-kylin git commit: KYLIN-780, query compile pass

Posted by li...@apache.org.
KYLIN-780, query compile pass


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/5d94f1aa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/5d94f1aa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/5d94f1aa

Branch: refs/heads/KYLIN-780
Commit: 5d94f1aa9b69a8eef2a9a14baa09474b7305dfd5
Parents: 650f81a
Author: Li, Yang <ya...@ebay.com>
Authored: Tue Jun 16 17:49:05 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Thu Jun 18 15:16:49 2015 +0800

----------------------------------------------------------------------
 .../hydromatic/optiq/runtime/SqlFunctions.java  | 1705 ------
 .../adapter/enumerable/EnumerableJoin.java      |  187 +
 .../apache/calcite/runtime/SqlFunctions.java    | 1261 +++++
 .../calcite/sql2rel/SqlToRelConverter.java      | 4978 ++++++++++++++++++
 .../eigenbase/sql2rel/SqlToRelConverter.java    | 4795 -----------------
 .../main/java/org/apache/kylin/jdbc/Driver.java |   12 +-
 .../apache/kylin/jdbc/KylinConnectionImpl.java  |   12 +-
 .../org/apache/kylin/jdbc/KylinEnumerator.java  |    2 +-
 .../apache/kylin/jdbc/KylinJdbc41Factory.java   |   20 +-
 .../org/apache/kylin/jdbc/KylinMetaImpl.java    |   24 +-
 .../org/apache/kylin/jdbc/KylinPrepare.java     |   10 +-
 .../org/apache/kylin/jdbc/KylinResultSet.java   |    6 +-
 .../apache/kylin/jdbc/KylinStatementImpl.java   |    6 +-
 .../org/apache/kylin/jdbc/stub/DataSet.java     |    4 +-
 .../org/apache/kylin/jdbc/stub/KylinClient.java |    8 +-
 .../apache/kylin/jdbc/stub/RemoteClient.java    |    2 +-
 .../kylin/query/enumerator/CubeEnumerator.java  |   15 +-
 .../kylin/query/enumerator/HiveEnumerator.java  |    8 +-
 .../query/enumerator/LookupTableEnumerator.java |    5 +-
 .../kylin/query/enumerator/OLAPQuery.java       |   12 +-
 .../kylin/query/optrule/OLAPAggregateRule.java  |   22 +-
 .../kylin/query/optrule/OLAPFilterRule.java     |   18 +-
 .../kylin/query/optrule/OLAPJoinRule.java       |   28 +-
 .../kylin/query/optrule/OLAPLimitRule.java      |   19 +-
 .../kylin/query/optrule/OLAPProjectRule.java    |   18 +-
 .../kylin/query/optrule/OLAPSortRule.java       |   19 +-
 .../optrule/OLAPToEnumerableConverterRule.java  |   13 +-
 .../kylin/query/relnode/OLAPAggregateRel.java   |  123 +-
 .../apache/kylin/query/relnode/OLAPContext.java |    8 +-
 .../kylin/query/relnode/OLAPFilterRel.java      |   94 +-
 .../apache/kylin/query/relnode/OLAPJoinRel.java |   65 +-
 .../kylin/query/relnode/OLAPLimitRel.java       |   44 +-
 .../kylin/query/relnode/OLAPProjectRel.java     |   82 +-
 .../org/apache/kylin/query/relnode/OLAPRel.java |   16 +-
 .../apache/kylin/query/relnode/OLAPSortRel.java |   49 +-
 .../kylin/query/relnode/OLAPTableScan.java      |   70 +-
 .../relnode/OLAPToEnumerableConverter.java      |   40 +-
 .../apache/kylin/query/schema/OLAPSchema.java   |    6 +-
 .../kylin/query/schema/OLAPSchemaFactory.java   |   22 +-
 .../apache/kylin/query/schema/OLAPTable.java    |   48 +-
 .../apache/kylin/query/sqlfunc/QuarterBase.java |   38 -
 .../apache/kylin/query/sqlfunc/QuarterFunc.java |   34 -
 .../apache/kylin/rest/service/QueryService.java |   16 +-
 43 files changed, 6868 insertions(+), 7096 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java b/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java
deleted file mode 100644
index db99c83..0000000
--- a/atopcalcite/src/main/java/net/hydromatic/optiq/runtime/SqlFunctions.java
+++ /dev/null
@@ -1,1705 +0,0 @@
-/*
- * OVERRIDE POINT:
- * - divide(BigDecimal,BigDecimal), was `b0.divide(b1)`, now `b0.divide(b1, MathContext.DECIMAL64);` 
- */
-
-/*
- * 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 net.hydromatic.optiq.runtime;
-
-import net.hydromatic.avatica.ByteString;
-
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Linq4j;
-import net.hydromatic.linq4j.expressions.Primitive;
-import net.hydromatic.linq4j.function.Deterministic;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.linq4j.function.NonDeterministic;
-
-import net.hydromatic.optiq.DataContext;
-
-import org.eigenbase.util14.DateTimeUtil;
-
-import java.math.*;
-import java.sql.SQLException;
-import java.sql.Timestamp;
-import java.text.DecimalFormat;
-import java.util.*;
-import java.util.regex.Pattern;
-
-/**
- * Helper methods to implement SQL functions in generated code.
- *
- * <p>Not present: and, or, not (builtin operators are better, because they
- * use lazy evaluation. Implementations do not check for null values; the
- * calling code must do that.</p>
- *
- * <p>Many of the functions do not check for null values. This is intentional.
- * If null arguments are possible, the code-generation framework checks for
- * nulls before calling the functions.</p>
- */
-@SuppressWarnings("UnnecessaryUnboxing")
-@Deterministic
-public class SqlFunctions {
-  private static final DecimalFormat DOUBLE_FORMAT =
-      new DecimalFormat("0.0E0");
-
-  /** The julian date of the epoch, 1970-01-01. */
-  public static final int EPOCH_JULIAN = 2440588;
-
-  private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
-
-  private static final Function1<List<Object>, Enumerable<Object>>
-  LIST_AS_ENUMERABLE =
-      new Function1<List<Object>, Enumerable<Object>>() {
-        public Enumerable<Object> apply(List<Object> list) {
-          return Linq4j.asEnumerable(list);
-        }
-      };
-
-  private SqlFunctions() {
-  }
-
-  /** SQL SUBSTRING(string FROM ... FOR ...) function. */
-  public static String substring(String s, int from, int for_) {
-    return s.substring(from - 1, Math.min(from - 1 + for_, s.length()));
-  }
-
-  /** SQL SUBSTRING(string FROM ...) function. */
-  public static String substring(String s, int from) {
-    return s.substring(from - 1);
-  }
-
-  /** SQL UPPER(string) function. */
-  public static String upper(String s) {
-    return s.toUpperCase();
-  }
-
-  /** SQL LOWER(string) function. */
-  public static String lower(String s) {
-    return s.toLowerCase();
-  }
-
-  /** SQL INITCAP(string) function. */
-  public static String initcap(String s) {
-    // Assumes Alpha as [A-Za-z0-9]
-    // white space is treated as everything else.
-    final int len = s.length();
-    boolean start = true;
-    final StringBuilder newS = new StringBuilder();
-
-    for (int i = 0; i < len; i++) {
-      char curCh = s.charAt(i);
-      final int c = (int) curCh;
-      if (start) {  // curCh is whitespace or first character of word.
-        if (c > 47 && c < 58) { // 0-9
-          start = false;
-        } else if (c > 64 && c < 91) {  // A-Z
-          start = false;
-        } else if (c > 96 && c < 123) {  // a-z
-          start = false;
-          curCh = (char) (c - 32); // Uppercase this character
-        }
-        // else {} whitespace
-      } else {  // Inside of a word or white space after end of word.
-        if (c > 47 && c < 58) { // 0-9
-          // noop
-        } else if (c > 64 && c < 91) {  // A-Z
-          curCh = (char) (c + 32); // Lowercase this character
-        } else if (c > 96 && c < 123) {  // a-z
-          // noop
-        } else { // whitespace
-          start = true;
-        }
-      }
-      newS.append(curCh);
-    } // for each character in s
-    return newS.toString();
-  }
-
-  /** SQL CHARACTER_LENGTH(string) function. */
-  public static int charLength(String s) {
-    return s.length();
-  }
-
-  /** SQL {@code string || string} operator. */
-  public static String concat(String s0, String s1) {
-    return s0 + s1;
-  }
-
-  /** SQL {@code binary || binary} operator. */
-  public static ByteString concat(ByteString s0, ByteString s1) {
-    return s0.concat(s1);
-  }
-
-  /** SQL {@code RTRIM} function applied to string. */
-  public static String rtrim(String s) {
-    return trim_(s, false, true, ' ');
-  }
-
-  /** SQL {@code LTRIM} function. */
-  public static String ltrim(String s) {
-    return trim_(s, true, false, ' ');
-  }
-
-  /** SQL {@code TRIM(... seek FROM s)} function. */
-  public static String trim(boolean leading, boolean trailing, String seek,
-      String s) {
-    return trim_(s, leading, trailing, seek.charAt(0));
-  }
-
-  /** SQL {@code TRIM} function. */
-  private static String trim_(String s, boolean left, boolean right, char c) {
-    int j = s.length();
-    if (right) {
-      for (;;) {
-        if (j == 0) {
-          return "";
-        }
-        if (s.charAt(j - 1) != c) {
-          break;
-        }
-        --j;
-      }
-    }
-    int i = 0;
-    if (left) {
-      for (;;) {
-        if (i == j) {
-          return "";
-        }
-        if (s.charAt(i) != c) {
-          break;
-        }
-        ++i;
-      }
-    }
-    return s.substring(i, j);
-  }
-
-  /** SQL {@code TRIM} function applied to binary string. */
-  public static ByteString trim(ByteString s) {
-    return trim_(s, true, true);
-  }
-
-  /** Helper for CAST. */
-  public static ByteString rtrim(ByteString s) {
-    return trim_(s, false, true);
-  }
-
-  /** SQL {@code TRIM} function applied to binary string. */
-  private static ByteString trim_(ByteString s, boolean left, boolean right) {
-    int j = s.length();
-    if (right) {
-      for (;;) {
-        if (j == 0) {
-          return ByteString.EMPTY;
-        }
-        if (s.byteAt(j - 1) != 0) {
-          break;
-        }
-        --j;
-      }
-    }
-    int i = 0;
-    if (left) {
-      for (;;) {
-        if (i == j) {
-          return ByteString.EMPTY;
-        }
-        if (s.byteAt(i) != 0) {
-          break;
-        }
-        ++i;
-      }
-    }
-    return s.substring(i, j);
-  }
-
-  /** SQL {@code OVERLAY} function. */
-  public static String overlay(String s, String r, int start) {
-    if (s == null || r == null) {
-      return null;
-    }
-    return s.substring(0, start - 1)
-        + r
-        + s.substring(start - 1 + r.length());
-  }
-
-  /** SQL {@code OVERLAY} function. */
-  public static String overlay(String s, String r, int start, int length) {
-    if (s == null || r == null) {
-      return null;
-    }
-    return s.substring(0, start - 1)
-        + r
-        + s.substring(start - 1 + length);
-  }
-
-  /** SQL {@code OVERLAY} function applied to binary strings. */
-  public static ByteString overlay(ByteString s, ByteString r, int start) {
-    if (s == null || r == null) {
-      return null;
-    }
-    return s.substring(0, start - 1)
-           .concat(r)
-           .concat(s.substring(start - 1 + r.length()));
-  }
-
-  /** SQL {@code OVERLAY} function applied to binary strings. */
-  public static ByteString overlay(ByteString s, ByteString r, int start,
-      int length) {
-    if (s == null || r == null) {
-      return null;
-    }
-    return s.substring(0, start - 1)
-           .concat(r)
-           .concat(s.substring(start - 1 + length));
-  }
-
-  /** SQL {@code LIKE} function. */
-  public static boolean like(String s, String pattern) {
-    final String regex = Like.sqlToRegexLike(pattern, null);
-    return Pattern.matches(regex, s);
-  }
-
-  /** SQL {@code LIKE} function with escape. */
-  public static boolean like(String s, String pattern, String escape) {
-    final String regex = Like.sqlToRegexLike(pattern, escape);
-    return Pattern.matches(regex, s);
-  }
-
-  /** SQL {@code SIMILAR} function. */
-  public static boolean similar(String s, String pattern) {
-    final String regex = Like.sqlToRegexSimilar(pattern, null);
-    return Pattern.matches(regex, s);
-  }
-
-  /** SQL {@code SIMILAR} function with escape. */
-  public static boolean similar(String s, String pattern, String escape) {
-    final String regex = Like.sqlToRegexSimilar(pattern, escape);
-    return Pattern.matches(regex, s);
-  }
-
-  // =
-
-  /** SQL = operator applied to Object values (including String; neither
-   * side may be null). */
-  public static boolean eq(Object b0, Object b1) {
-    return b0.equals(b1);
-  }
-
-  /** SQL = operator applied to BigDecimal values (neither may be null). */
-  public static boolean eq(BigDecimal b0, BigDecimal b1) {
-    return b0.stripTrailingZeros().equals(b1.stripTrailingZeros());
-  }
-
-  // <>
-
-  /** SQL &lt;&gt; operator applied to Object values (including String;
-   * neither side may be null). */
-  public static boolean ne(Object b0, Object b1) {
-    return !b0.equals(b1);
-  }
-
-  /** SQL &lt;&gt; operator applied to BigDecimal values. */
-  public static boolean ne(BigDecimal b0, BigDecimal b1) {
-    return b0.compareTo(b1) != 0;
-  }
-
-  // <
-
-  /** SQL &lt; operator applied to boolean values. */
-  public static boolean lt(boolean b0, boolean b1) {
-    return compare(b0, b1) < 0;
-  }
-
-  /** SQL &lt; operator applied to String values. */
-  public static boolean lt(String b0, String b1) {
-    return b0.compareTo(b1) < 0;
-  }
-
-  /** SQL &lt; operator applied to ByteString values. */
-  public static boolean lt(ByteString b0, ByteString b1) {
-    return b0.compareTo(b1) < 0;
-  }
-
-  /** SQL &lt; operator applied to BigDecimal values. */
-  public static boolean lt(BigDecimal b0, BigDecimal b1) {
-    return b0.compareTo(b1) < 0;
-  }
-
-  // <=
-
-  /** SQL &le; operator applied to boolean values. */
-  public static boolean le(boolean b0, boolean b1) {
-    return compare(b0, b1) <= 0;
-  }
-
-  /** SQL &le; operator applied to String values. */
-  public static boolean le(String b0, String b1) {
-    return b0.compareTo(b1) <= 0;
-  }
-
-  /** SQL &le; operator applied to ByteString values. */
-  public static boolean le(ByteString b0, ByteString b1) {
-    return b0.compareTo(b1) <= 0;
-  }
-
-  /** SQL &le; operator applied to BigDecimal values. */
-  public static boolean le(BigDecimal b0, BigDecimal b1) {
-    return b0.compareTo(b1) <= 0;
-  }
-
-  // >
-
-  /** SQL &gt; operator applied to boolean values. */
-  public static boolean gt(boolean b0, boolean b1) {
-    return compare(b0, b1) > 0;
-  }
-
-  /** SQL &gt; operator applied to String values. */
-  public static boolean gt(String b0, String b1) {
-    return b0.compareTo(b1) > 0;
-  }
-
-  /** SQL &gt; operator applied to ByteString values. */
-  public static boolean gt(ByteString b0, ByteString b1) {
-    return b0.compareTo(b1) > 0;
-  }
-
-  /** SQL &gt; operator applied to BigDecimal values. */
-  public static boolean gt(BigDecimal b0, BigDecimal b1) {
-    return b0.compareTo(b1) > 0;
-  }
-
-  // >=
-
-  /** SQL &ge; operator applied to boolean values. */
-  public static boolean ge(boolean b0, boolean b1) {
-    return compare(b0, b1) >= 0;
-  }
-
-  /** SQL &ge; operator applied to String values. */
-  public static boolean ge(String b0, String b1) {
-    return b0.compareTo(b1) >= 0;
-  }
-
-  /** SQL &ge; operator applied to ByteString values. */
-  public static boolean ge(ByteString b0, ByteString b1) {
-    return b0.compareTo(b1) >= 0;
-  }
-
-  /** SQL &ge; operator applied to BigDecimal values. */
-  public static boolean ge(BigDecimal b0, BigDecimal b1) {
-    return b0.compareTo(b1) >= 0;
-  }
-
-  // +
-
-  /** SQL <code>+</code> operator applied to int values. */
-  public static int plus(int b0, int b1) {
-    return b0 + b1;
-  }
-
-  /** SQL <code>+</code> operator applied to int values; left side may be
-   * null. */
-  public static Integer plus(Integer b0, int b1) {
-    return b0 == null ? null : (b0 + b1);
-  }
-
-  /** SQL <code>+</code> operator applied to int values; right side may be
-   * null. */
-  public static Integer plus(int b0, Integer b1) {
-    return b1 == null ? null : (b0 + b1);
-  }
-
-  /** SQL <code>+</code> operator applied to nullable int values. */
-  public static Integer plus(Integer b0, Integer b1) {
-    return (b0 == null || b1 == null) ? null : (b0 + b1);
-  }
-
-  /** SQL <code>+</code> operator applied to nullable long and int values. */
-  public static Long plus(Long b0, Integer b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() + b1.longValue());
-  }
-
-  /** SQL <code>+</code> operator applied to nullable int and long values. */
-  public static Long plus(Integer b0, Long b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() + b1.longValue());
-  }
-
-  /** SQL <code>+</code> operator applied to BigDecimal values. */
-  public static BigDecimal plus(BigDecimal b0, BigDecimal b1) {
-    return (b0 == null || b1 == null) ? null : b0.add(b1);
-  }
-
-  // -
-
-  /** SQL <code>-</code> operator applied to int values. */
-  public static int minus(int b0, int b1) {
-    return b0 - b1;
-  }
-
-  /** SQL <code>-</code> operator applied to int values; left side may be
-   * null. */
-  public static Integer minus(Integer b0, int b1) {
-    return b0 == null ? null : (b0 - b1);
-  }
-
-  /** SQL <code>-</code> operator applied to int values; right side may be
-   * null. */
-  public static Integer minus(int b0, Integer b1) {
-    return b1 == null ? null : (b0 - b1);
-  }
-
-  /** SQL <code>-</code> operator applied to nullable int values. */
-  public static Integer minus(Integer b0, Integer b1) {
-    return (b0 == null || b1 == null) ? null : (b0 - b1);
-  }
-
-  /** SQL <code>-</code> operator applied to nullable long and int values. */
-  public static Long minus(Long b0, Integer b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() - b1.longValue());
-  }
-
-  /** SQL <code>-</code> operator applied to nullable int and long values. */
-  public static Long minus(Integer b0, Long b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() - b1.longValue());
-  }
-
-  /** SQL <code>-</code> operator applied to BigDecimal values. */
-  public static BigDecimal minus(BigDecimal b0, BigDecimal b1) {
-    return (b0 == null || b1 == null) ? null : b0.subtract(b1);
-  }
-
-  // /
-
-  /** SQL <code>/</code> operator applied to int values. */
-  public static int divide(int b0, int b1) {
-    return b0 / b1;
-  }
-
-  /** SQL <code>/</code> operator applied to int values; left side may be
-   * null. */
-  public static Integer divide(Integer b0, int b1) {
-    return b0 == null ? null : (b0 / b1);
-  }
-
-  /** SQL <code>/</code> operator applied to int values; right side may be
-   * null. */
-  public static Integer divide(int b0, Integer b1) {
-    return b1 == null ? null : (b0 / b1);
-  }
-
-  /** SQL <code>/</code> operator applied to nullable int values. */
-  public static Integer divide(Integer b0, Integer b1) {
-    return (b0 == null || b1 == null) ? null : (b0 / b1);
-  }
-
-  /** SQL <code>/</code> operator applied to nullable long and int values. */
-  public static Long divide(Long b0, Integer b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() / b1.longValue());
-  }
-
-  /** SQL <code>/</code> operator applied to nullable int and long values. */
-  public static Long divide(Integer b0, Long b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() / b1.longValue());
-  }
-
-  /** SQL <code>/</code> operator applied to BigDecimal values. */
-  public static BigDecimal divide(BigDecimal b0, BigDecimal b1) {
-    // OVERRIDE POINT
-    return (b0 == null || b1 == null) ? null : b0.divide(b1, MathContext.DECIMAL64);
-  }
-
-  // *
-
-  /** SQL <code>*</code> operator applied to int values. */
-  public static int multiply(int b0, int b1) {
-    return b0 * b1;
-  }
-
-  /** SQL <code>*</code> operator applied to int values; left side may be
-   * null. */
-  public static Integer multiply(Integer b0, int b1) {
-    return b0 == null ? null : (b0 * b1);
-  }
-
-  /** SQL <code>*</code> operator applied to int values; right side may be
-   * null. */
-  public static Integer multiply(int b0, Integer b1) {
-    return b1 == null ? null : (b0 * b1);
-  }
-
-  /** SQL <code>*</code> operator applied to nullable int values. */
-  public static Integer multiply(Integer b0, Integer b1) {
-    return (b0 == null || b1 == null) ? null : (b0 * b1);
-  }
-
-  /** SQL <code>*</code> operator applied to nullable long and int values. */
-  public static Long multiply(Long b0, Integer b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() * b1.longValue());
-  }
-
-  /** SQL <code>*</code> operator applied to nullable int and long values. */
-  public static Long multiply(Integer b0, Long b1) {
-    return (b0 == null || b1 == null)
-        ? null
-        : (b0.longValue() * b1.longValue());
-  }
-
-  /** SQL <code>*</code> operator applied to BigDecimal values. */
-  public static BigDecimal multiply(BigDecimal b0, BigDecimal b1) {
-    return (b0 == null || b1 == null) ? null : b0.multiply(b1);
-  }
-
-  // EXP
-
-  /** SQL <code>EXP</code> operator applied to double values. */
-  public static double exp(double b0) {
-    return Math.exp(b0);
-  }
-
-  public static double exp(long b0) {
-    return Math.exp(b0);
-  }
-
-  // POWER
-
-  /** SQL <code>POWER</code> operator applied to double values. */
-  public static double power(double b0, double b1) {
-    return Math.pow(b0, b1);
-  }
-
-  public static double power(long b0, long b1) {
-    return Math.pow(b0, b1);
-  }
-
-  public static double power(long b0, BigDecimal b1) {
-    return Math.pow(b0, b1.doubleValue());
-  }
-
-  // LN
-
-  /** SQL {@code LN(number)} function applied to double values. */
-  public static double ln(double d) {
-    return Math.log(d);
-  }
-
-  /** SQL {@code LN(number)} function applied to long values. */
-  public static double ln(long b0) {
-    return Math.log(b0);
-  }
-
-  /** SQL {@code LN(number)} function applied to BigDecimal values. */
-  public static double ln(BigDecimal d) {
-    return Math.log(d.doubleValue());
-  }
-
-  // LOG10
-
-  /** SQL <code>LOG10(numeric)</code> operator applied to double values. */
-  public static double log10(double b0) {
-    return Math.log10(b0);
-  }
-
-  /** SQL {@code LOG10(number)} function applied to long values. */
-  public static double log10(long b0) {
-    return Math.log10(b0);
-  }
-
-  /** SQL {@code LOG10(number)} function applied to BigDecimal values. */
-  public static double log10(BigDecimal d) {
-    return Math.log10(d.doubleValue());
-  }
-
-  // MOD
-
-  /** SQL <code>MOD</code> operator applied to byte values. */
-  public static byte mod(byte b0, byte b1) {
-    return (byte) (b0 % b1);
-  }
-
-  /** SQL <code>MOD</code> operator applied to short values. */
-  public static short mod(short b0, short b1) {
-    return (short) (b0 % b1);
-  }
-
-  /** SQL <code>MOD</code> operator applied to int values. */
-  public static int mod(int b0, int b1) {
-    return b0 % b1;
-  }
-
-  /** SQL <code>MOD</code> operator applied to long values. */
-  public static long mod(long b0, long b1) {
-    return b0 % b1;
-  }
-
-  // temporary
-  public static BigDecimal mod(BigDecimal b0, int b1) {
-    return mod(b0, BigDecimal.valueOf(b1));
-  }
-
-  // temporary
-  public static int mod(int b0, BigDecimal b1) {
-    return mod(b0, b1.intValue());
-  }
-
-  public static BigDecimal mod(BigDecimal b0, BigDecimal b1) {
-    final BigDecimal[] bigDecimals = b0.divideAndRemainder(b1);
-    return bigDecimals[1];
-  }
-
-  // ABS
-
-  /** SQL <code>ABS</code> operator applied to byte values. */
-  public static byte abs(byte b0) {
-    return (byte) Math.abs(b0);
-  }
-
-  /** SQL <code>ABS</code> operator applied to short values. */
-  public static short abs(short b0) {
-    return (short) Math.abs(b0);
-  }
-
-  /** SQL <code>ABS</code> operator applied to int values. */
-  public static int abs(int b0) {
-    return Math.abs(b0);
-  }
-
-  /** SQL <code>ABS</code> operator applied to long values. */
-  public static long abs(long b0) {
-    return Math.abs(b0);
-  }
-
-  /** SQL <code>ABS</code> operator applied to float values. */
-  public static float abs(float b0) {
-    return Math.abs(b0);
-  }
-
-  /** SQL <code>ABS</code> operator applied to double values. */
-  public static double abs(double b0) {
-    return Math.abs(b0);
-  }
-
-  /** SQL <code>ABS</code> operator applied to BigDecimal values. */
-  public static BigDecimal abs(BigDecimal b0) {
-    return b0.abs();
-  }
-
-  // Helpers
-
-  /** Helper for implementing MIN. Somewhat similar to LEAST operator. */
-  public static <T extends Comparable<T>> T lesser(T b0, T b1) {
-    return b0 == null || b0.compareTo(b1) > 0 ? b1 : b0;
-  }
-
-  /** LEAST operator. */
-  public static <T extends Comparable<T>> T least(T b0, T b1) {
-    return b0 == null || b1 != null && b0.compareTo(b1) > 0 ? b1 : b0;
-  }
-
-  public static boolean greater(boolean b0, boolean b1) {
-    return b0 || b1;
-  }
-
-  public static boolean lesser(boolean b0, boolean b1) {
-    return b0 && b1;
-  }
-
-  public static byte greater(byte b0, byte b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static byte lesser(byte b0, byte b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  public static char greater(char b0, char b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static char lesser(char b0, char b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  public static short greater(short b0, short b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static short lesser(short b0, short b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  public static int greater(int b0, int b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static int lesser(int b0, int b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  public static long greater(long b0, long b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static long lesser(long b0, long b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  public static float greater(float b0, float b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static float lesser(float b0, float b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  public static double greater(double b0, double b1) {
-    return b0 > b1 ? b0 : b1;
-  }
-
-  public static double lesser(double b0, double b1) {
-    return b0 > b1 ? b1 : b0;
-  }
-
-  /** Helper for implementing MAX. Somewhat similar to GREATEST operator. */
-  public static <T extends Comparable<T>> T greater(T b0, T b1) {
-    return b0 == null || b0.compareTo(b1) < 0 ? b1 : b0;
-  }
-
-  /** GREATEST operator. */
-  public static <T extends Comparable<T>> T greatest(T b0, T b1) {
-    return b0 == null || b1 != null && b0.compareTo(b1) < 0 ? b1 : b0;
-  }
-
-  /** Boolean comparison. */
-  public static int compare(boolean x, boolean y) {
-    return x == y ? 0 : x ? 1 : -1;
-  }
-
-  /** CAST(FLOAT AS VARCHAR). */
-  public static String toString(float x) {
-    if (x == 0) {
-      return "0E0";
-    }
-    BigDecimal bigDecimal =
-        new BigDecimal(x, MathContext.DECIMAL32).stripTrailingZeros();
-    final String s = bigDecimal.toString();
-    return s.replaceAll("0*E", "E").replace("E+", "E");
-  }
-
-  /** CAST(DOUBLE AS VARCHAR). */
-  public static String toString(double x) {
-    if (x == 0) {
-      return "0E0";
-    }
-    BigDecimal bigDecimal =
-        new BigDecimal(x, MathContext.DECIMAL64).stripTrailingZeros();
-    final String s = bigDecimal.toString();
-    return s.replaceAll("0*E", "E").replace("E+", "E");
-  }
-
-  /** CAST(DECIMAL AS VARCHAR). */
-  public static String toString(BigDecimal x) {
-    final String s = x.toString();
-    if (s.startsWith("0")) {
-      // we want ".1" not "0.1"
-      return s.substring(1);
-    } else if (s.startsWith("-0")) {
-      // we want "-.1" not "-0.1"
-      return "-" + s.substring(2);
-    } else {
-      return s;
-    }
-  }
-
-  /** CAST(BOOLEAN AS VARCHAR). */
-  public static String toString(boolean x) {
-    // Boolean.toString returns lower case -- no good.
-    return x ? "TRUE" : "FALSE";
-  }
-
-  @NonDeterministic
-  private static Object cannotConvert(Object o, Class toType) {
-    throw new RuntimeException("Cannot convert " + o + " to " + toType);
-  }
-
-  /** CAST(VARCHAR AS BOOLEAN). */
-  public static boolean toBoolean(String s) {
-    s = trim_(s, true, true, ' ');
-    if (s.equalsIgnoreCase("TRUE")) {
-      return true;
-    } else if (s.equalsIgnoreCase("FALSE")) {
-      return false;
-    } else {
-      throw new RuntimeException("Invalid character for cast");
-    }
-  }
-
-  public static boolean toBoolean(Number number) {
-    return !number.equals(0);
-  }
-
-  public static boolean toBoolean(Object o) {
-    return o instanceof Boolean ? (Boolean) o
-        : o instanceof Number ? toBoolean((Number) o)
-        : o instanceof String ? toBoolean((String) o)
-        : (Boolean) cannotConvert(o, boolean.class);
-  }
-
-  // Don't need parseByte etc. - Byte.parseByte is sufficient.
-
-  public static byte toByte(Object o) {
-    return o instanceof Byte ? (Byte) o
-        : o instanceof Number ? toByte((Number) o)
-        : Byte.parseByte(o.toString());
-  }
-
-  public static byte toByte(Number number) {
-    return number.byteValue();
-  }
-
-  public static char toChar(String s) {
-    return s.charAt(0);
-  }
-
-  public static Character toCharBoxed(String s) {
-    return s.charAt(0);
-  }
-
-  public static short toShort(String s) {
-    return Short.parseShort(s.trim());
-  }
-
-  public static short toShort(Number number) {
-    return number.shortValue();
-  }
-
-  public static short toShort(Object o) {
-    return o instanceof Short ? (Short) o
-        : o instanceof Number ? toShort((Number) o)
-        : o instanceof String ? toShort((String) o)
-        : (Short) cannotConvert(o, short.class);
-  }
-
-  public static int toInt(java.util.Date v) {
-    return toInt(v, LOCAL_TZ);
-  }
-
-  public static int toInt(java.util.Date v, TimeZone timeZone) {
-    return (int) (toLong(v, timeZone)  / DateTimeUtil.MILLIS_PER_DAY);
-  }
-
-  public static Integer toIntOptional(java.util.Date v) {
-    return v == null ? null : toInt(v);
-  }
-
-  public static Integer toIntOptional(java.util.Date v, TimeZone timeZone) {
-    return v == null
-        ? null
-        : toInt(v, timeZone);
-  }
-
-  public static long toLong(Date v) {
-    return toLong(v, LOCAL_TZ);
-  }
-
-  public static int toInt(java.sql.Time v) {
-    return (int) (toLong(v) % DateTimeUtil.MILLIS_PER_DAY);
-  }
-
-  public static Integer toIntOptional(java.sql.Time v) {
-    return v == null ? null : toInt(v);
-  }
-
-  public static int toInt(String s) {
-    return Integer.parseInt(s.trim());
-  }
-
-  public static int toInt(Number number) {
-    return number.intValue();
-  }
-
-  public static int toInt(Object o) {
-    return o instanceof Integer ? (Integer) o
-        : o instanceof Number ? toInt((Number) o)
-        : o instanceof String ? toInt((String) o)
-        : (Integer) cannotConvert(o, int.class);
-  }
-
-  public static long toLong(Timestamp v) {
-    return toLong(v, LOCAL_TZ);
-  }
-
-  // mainly intended for java.sql.Timestamp but works for other dates also
-  public static long toLong(java.util.Date v, TimeZone timeZone) {
-    final long time = v.getTime();
-    return time + timeZone.getOffset(time);
-  }
-
-  // mainly intended for java.sql.Timestamp but works for other dates also
-  public static Long toLongOptional(java.util.Date v) {
-    return v == null ? null : toLong(v, LOCAL_TZ);
-  }
-
-  public static Long toLongOptional(Timestamp v, TimeZone timeZone) {
-    if (v == null) {
-      return null;
-    }
-    return toLong(v, LOCAL_TZ);
-  }
-
-  public static long toLong(String s) {
-    if (s.startsWith("199") && s.contains(":")) {
-      return Timestamp.valueOf(s).getTime();
-    }
-    return Long.parseLong(s.trim());
-  }
-
-  public static long toLong(Number number) {
-    return number.longValue();
-  }
-
-  public static long toLong(Object o) {
-    return o instanceof Long ? (Long) o
-        : o instanceof Number ? toLong((Number) o)
-        : o instanceof String ? toLong((String) o)
-        : (Long) cannotConvert(o, long.class);
-  }
-
-  public static float toFloat(String s) {
-    return Float.parseFloat(s.trim());
-  }
-
-  public static float toFloat(Number number) {
-    return number.floatValue();
-  }
-
-  public static float toFloat(Object o) {
-    return o instanceof Float ? (Float) o
-        : o instanceof Number ? toFloat((Number) o)
-            : o instanceof String ? toFloat((String) o)
-                : (Float) cannotConvert(o, float.class);
-  }
-
-  public static double toDouble(String s) {
-    return Double.parseDouble(s.trim());
-  }
-
-  public static double toDouble(Number number) {
-    return number.doubleValue();
-  }
-
-  public static double toDouble(Object o) {
-    return o instanceof Double ? (Double) o
-        : o instanceof Number ? toDouble((Number) o)
-        : o instanceof String ? toDouble((String) o)
-        : (Double) cannotConvert(o, double.class);
-  }
-
-  public static BigDecimal toBigDecimal(String s) {
-    return new BigDecimal(s.trim());
-  }
-
-  public static BigDecimal toBigDecimal(Number number) {
-    // There are some values of "long" that cannot be represented as "double".
-    // Not so "int". If it isn't a long, go straight to double.
-    return number instanceof BigDecimal ? (BigDecimal) number
-        : number instanceof BigInteger ? new BigDecimal((BigInteger) number)
-        : number instanceof Long ? new BigDecimal(number.longValue())
-        : new BigDecimal(number.doubleValue());
-  }
-
-  public static BigDecimal toBigDecimal(Object o) {
-    return o instanceof Number ? toBigDecimal((Number) o)
-        : toBigDecimal(o.toString());
-  }
-
-  // Don't need shortValueOf etc. - Short.valueOf is sufficient.
-
-  /** Helper for CAST(... AS VARCHAR(maxLength)). */
-  public static String truncate(String s, int maxLength) {
-    return s == null ? null
-        : s.length() > maxLength ? s.substring(0, maxLength)
-        : s;
-  }
-
-  /** Helper for CAST(... AS VARBINARY(maxLength)). */
-  public static ByteString truncate(ByteString s, int maxLength) {
-    return s == null ? null
-        : s.length() > maxLength ? s.substring(0, maxLength)
-        : s;
-  }
-
-  /** SQL {@code POSITION(seek IN string)} function. */
-  public static int position(String seek, String s) {
-    return s.indexOf(seek) + 1;
-  }
-
-  /** SQL {@code POSITION(seek IN string)} function. */
-  public static int position(ByteString seek, ByteString s) {
-    return s.indexOf(seek) + 1;
-  }
-
-  /** Cheap, unsafe, long power. power(2, 3) returns 8. */
-  public static long powerX(long a, long b) {
-    long x = 1;
-    while (b > 0) {
-      x *= a;
-      --b;
-    }
-    return x;
-  }
-
-  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-  public static long round(long v, long x) {
-    return truncate(v + x / 2, x);
-  }
-
-  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-  public static long truncate(long v, long x) {
-    long remainder = v % x;
-    if (remainder < 0) {
-      remainder += x;
-    }
-    return v - remainder;
-  }
-
-  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-  public static int round(int v, int x) {
-    return truncate(v + x / 2, x);
-  }
-
-  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
-  public static int truncate(int v, int x) {
-    int remainder = v % x;
-    if (remainder < 0) {
-      remainder += x;
-    }
-    return v - remainder;
-  }
-
-  /** Helper for CAST({timestamp} AS VARCHAR(n)). */
-  public static String unixTimestampToString(long timestamp) {
-    final StringBuilder buf = new StringBuilder(17);
-    int date = (int) (timestamp / DateTimeUtil.MILLIS_PER_DAY);
-    int time = (int) (timestamp % DateTimeUtil.MILLIS_PER_DAY);
-    if (time < 0) {
-      --date;
-      time += DateTimeUtil.MILLIS_PER_DAY;
-    }
-    unixDateToString(buf, date);
-    buf.append(' ');
-    unixTimeToString(buf, time);
-    return buf.toString();
-  }
-
-  /** Helper for CAST({timestamp} AS VARCHAR(n)). */
-  public static String unixTimeToString(int time) {
-    final StringBuilder buf = new StringBuilder(8);
-    unixTimeToString(buf, time);
-    return buf.toString();
-  }
-
-  private static void unixTimeToString(StringBuilder buf, int time) {
-    int h = time / 3600000;
-    int time2 = time % 3600000;
-    int m = time2 / 60000;
-    int time3 = time2 % 60000;
-    int s = time3 / 1000;
-    int ms = time3 % 1000;
-    int2(buf, h);
-    buf.append(':');
-    int2(buf, m);
-    buf.append(':');
-    int2(buf, s);
-  }
-
-  /** SQL {@code CURRENT_TIMESTAMP} function. */
-  @NonDeterministic
-  public static long currentTimestamp(DataContext root) {
-    // Cast required for JDK 1.6.
-    return (Long) DataContext.Variable.CURRENT_TIMESTAMP.get(root);
-  }
-
-  /** SQL {@code CURRENT_TIME} function. */
-  @NonDeterministic
-  public static int currentTime(DataContext root) {
-    int time = (int) (currentTimestamp(root) % DateTimeUtil.MILLIS_PER_DAY);
-    if (time < 0) {
-      time += DateTimeUtil.MILLIS_PER_DAY;
-    }
-    return time;
-  }
-
-  /** SQL {@code CURRENT_DATE} function. */
-  @NonDeterministic
-  public static int currentDate(DataContext root) {
-    final long timestamp = currentTimestamp(root);
-    int date = (int) (timestamp / DateTimeUtil.MILLIS_PER_DAY);
-    final int time = (int) (timestamp % DateTimeUtil.MILLIS_PER_DAY);
-    if (time < 0) {
-      --date;
-    }
-    return date;
-  }
-
-  /** SQL {@code LOCAL_TIMESTAMP} function. */
-  @NonDeterministic
-  public static long localTimestamp(DataContext root) {
-    // Cast required for JDK 1.6.
-    return (Long) DataContext.Variable.LOCAL_TIMESTAMP.get(root);
-  }
-
-  /** SQL {@code LOCAL_TIME} function. */
-  @NonDeterministic
-  public static int localTime(DataContext root) {
-    return (int) (localTimestamp(root) % DateTimeUtil.MILLIS_PER_DAY);
-  }
-
-  private static void int2(StringBuilder buf, int i) {
-    buf.append((char) ('0' + (i / 10) % 10));
-    buf.append((char) ('0' + i % 10));
-  }
-
-  private static void int4(StringBuilder buf, int i) {
-    buf.append((char) ('0' + (i / 1000) % 10));
-    buf.append((char) ('0' + (i / 100) % 10));
-    buf.append((char) ('0' + (i / 10) % 10));
-    buf.append((char) ('0' + i % 10));
-  }
-
-  public static int dateStringToUnixDate(String s) {
-    int hyphen1 = s.indexOf('-');
-    int y;
-    int m;
-    int d;
-    if (hyphen1 < 0) {
-      y = Integer.parseInt(s.trim());
-      m = 1;
-      d = 1;
-    } else {
-      y = Integer.parseInt(s.substring(0, hyphen1).trim());
-      final int hyphen2 = s.indexOf('-', hyphen1 + 1);
-      if (hyphen2 < 0) {
-        m = Integer.parseInt(s.substring(hyphen1 + 1).trim());
-        d = 1;
-      } else {
-        m = Integer.parseInt(s.substring(hyphen1 + 1, hyphen2).trim());
-        d = Integer.parseInt(s.substring(hyphen2 + 1).trim());
-      }
-    }
-    return ymdToUnixDate(y, m, d);
-  }
-
-  public static int timeStringToUnixDate(String v) {
-    return timeStringToUnixDate(v, 0);
-  }
-
-  public static int timeStringToUnixDate(String v, int start) {
-    final int colon1 = v.indexOf(':', start);
-    int hour;
-    int minute;
-    int second;
-    int milli;
-    if (colon1 < 0) {
-      hour = Integer.parseInt(v.trim());
-      minute = 1;
-      second = 1;
-      milli = 0;
-    } else {
-      hour = Integer.parseInt(v.substring(start, colon1).trim());
-      final int colon2 = v.indexOf(':', colon1 + 1);
-      if (colon2 < 0) {
-        minute = Integer.parseInt(v.substring(colon1 + 1).trim());
-        second = 1;
-        milli = 0;
-      } else {
-        minute = Integer.parseInt(v.substring(colon1 + 1, colon2).trim());
-        int dot = v.indexOf('.', colon2);
-        if (dot < 0) {
-          second = Integer.parseInt(v.substring(colon2 + 1).trim());
-          milli = 0;
-        } else {
-          second = Integer.parseInt(v.substring(colon2 + 1, dot).trim());
-          milli = Integer.parseInt(v.substring(dot + 1).trim());
-        }
-      }
-    }
-    return hour * (int) DateTimeUtil.MILLIS_PER_HOUR
-        + minute * (int) DateTimeUtil.MILLIS_PER_MINUTE
-        + second * (int) DateTimeUtil.MILLIS_PER_SECOND
-        + milli;
-  }
-
-  public static long timestampStringToUnixDate(String s) {
-    final long d;
-    final long t;
-    s = s.trim();
-    int space = s.indexOf(' ');
-    if (space >= 0) {
-      d = dateStringToUnixDate(s.substring(0, space));
-      t = timeStringToUnixDate(s, space + 1);
-    } else {
-      d = dateStringToUnixDate(s);
-      t = 0;
-    }
-    return d * DateTimeUtil.MILLIS_PER_DAY + t;
-  }
-
-  /** Helper for CAST({date} AS VARCHAR(n)). */
-  public static String unixDateToString(int date) {
-    final StringBuilder buf = new StringBuilder(10);
-    unixDateToString(buf, date);
-    return buf.toString();
-  }
-
-  private static void unixDateToString(StringBuilder buf, int date) {
-    julianToString(buf, date + EPOCH_JULIAN);
-  }
-
-  private static void julianToString(StringBuilder buf, int julian) {
-    // this shifts the epoch back to astronomical year -4800 instead of the
-    // start of the Christian era in year AD 1 of the proleptic Gregorian
-    // calendar.
-    int j = julian + 32044;
-    int g = j / 146097;
-    int dg = j % 146097;
-    int c = (dg / 36524 + 1) * 3 / 4;
-    int dc = dg - c * 36524;
-    int b = dc / 1461;
-    int db = dc % 1461;
-    int a = (db / 365 + 1) * 3 / 4;
-    int da = db - a * 365;
-
-    // integer number of full years elapsed since March 1, 4801 BC
-    int y = g * 400 + c * 100 + b * 4 + a;
-    // integer number of full months elapsed since the last March 1
-    int m = (da * 5 + 308) / 153 - 2;
-    // number of days elapsed since day 1 of the month
-    int d = da - (m + 4) * 153 / 5 + 122;
-    int year = y - 4800 + (m + 2) / 12;
-    int month = (m + 2) % 12 + 1;
-    int day = d + 1;
-    int4(buf, year);
-    buf.append('-');
-    int2(buf, month);
-    buf.append('-');
-    int2(buf, day);
-  }
-
-  public static long unixDateExtract(TimeUnitRange range, long date) {
-    return julianExtract(range, (int) date + EPOCH_JULIAN);
-  }
-
-  private static int julianExtract(TimeUnitRange range, int julian) {
-    // this shifts the epoch back to astronomical year -4800 instead of the
-    // start of the Christian era in year AD 1 of the proleptic Gregorian
-    // calendar.
-    int j = julian + 32044;
-    int g = j / 146097;
-    int dg = j % 146097;
-    int c = (dg / 36524 + 1) * 3 / 4;
-    int dc = dg - c * 36524;
-    int b = dc / 1461;
-    int db = dc % 1461;
-    int a = (db / 365 + 1) * 3 / 4;
-    int da = db - a * 365;
-
-    // integer number of full years elapsed since March 1, 4801 BC
-    int y = g * 400 + c * 100 + b * 4 + a;
-    // integer number of full months elapsed since the last March 1
-    int m = (da * 5 + 308) / 153 - 2;
-    // number of days elapsed since day 1 of the month
-    int d = da - (m + 4) * 153 / 5 + 122;
-    int year = y - 4800 + (m + 2) / 12;
-    int month = (m + 2) % 12 + 1;
-    int day = d + 1;
-    switch (range) {
-    case YEAR:
-      return year;
-    case MONTH:
-      return month;
-    case DAY:
-      return day;
-    default:
-      throw new AssertionError(range);
-    }
-  }
-
-  public static int ymdToUnixDate(int year, int month, int day) {
-    final int julian = ymdToJulian(year, month, day);
-    return julian - EPOCH_JULIAN;
-  }
-
-  public static int ymdToJulian(int year, int month, int day) {
-    int a = (14 - month) / 12;
-    int y = year + 4800 - a;
-    int m = month + 12 * a - 3;
-    int j = day + (153 * m + 2) / 5
-        + 365 * y
-        + y / 4
-        - y / 100
-        + y / 400
-        - 32045;
-    if (j < 2299161) {
-      j = day + (153 * m + 2) / 5 + 365 * y + y / 4 - 32083;
-    }
-    return j;
-  }
-
-  public static String intervalYearMonthToString(int v, TimeUnitRange range) {
-    final StringBuilder buf = new StringBuilder();
-    if (v >= 0) {
-      buf.append('+');
-    } else {
-      buf.append('-');
-      v = -v;
-    }
-    final int y;
-    final int m;
-    switch (range) {
-    case YEAR:
-      v = roundUp(v, 12);
-      y = v / 12;
-      buf.append(y);
-      break;
-    case YEAR_TO_MONTH:
-      y = v / 12;
-      buf.append(y);
-      buf.append('-');
-      m = v % 12;
-      number(buf, m, 2);
-      break;
-    case MONTH:
-      m = v;
-      buf.append(m);
-      break;
-    default:
-      throw new AssertionError(range);
-    }
-    return buf.toString();
-  }
-
-  private static StringBuilder number(StringBuilder buf, int v, int n) {
-    for (int k = digitCount(v); k < n; k++) {
-      buf.append('0');
-    }
-    return buf.append(v);
-  }
-
-  public static int digitCount(int v) {
-    for (int n = 1;; n++) {
-      v /= 10;
-      if (v == 0) {
-        return n;
-      }
-    }
-  }
-
-  public static String intervalDayTimeToString(long v, TimeUnitRange range,
-      int scale) {
-    final StringBuilder buf = new StringBuilder();
-    if (v >= 0) {
-      buf.append('+');
-    } else {
-      buf.append('-');
-      v = -v;
-    }
-    final long ms;
-    final long s;
-    final long m;
-    final long h;
-    final long d;
-    switch (range) {
-    case DAY_TO_SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v % 60;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v % 24;
-      v /= 24;
-      d = v;
-      buf.append((int) d);
-      buf.append(' ');
-      number(buf, (int) h, 2);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      buf.append(':');
-      number(buf, (int) s, 2);
-      fraction(buf, scale, ms);
-      break;
-    case DAY_TO_MINUTE:
-      v = roundUp(v, 1000 * 60);
-      v /= 1000;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v % 24;
-      v /= 24;
-      d = v;
-      buf.append((int) d);
-      buf.append(' ');
-      number(buf, (int) h, 2);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      break;
-    case DAY_TO_HOUR:
-      v = roundUp(v, 1000 * 60 * 60);
-      v /= 1000;
-      v /= 60;
-      v /= 60;
-      h = v % 24;
-      v /= 24;
-      d = v;
-      buf.append((int) d);
-      buf.append(' ');
-      number(buf, (int) h, 2);
-      break;
-    case DAY:
-      v = roundUp(v, 1000 * 60 * 60 * 24);
-      d = v / (1000 * 60 * 60 * 24);
-      buf.append((int) d);
-      break;
-    case HOUR:
-      v = roundUp(v, 1000 * 60 * 60);
-      v /= 1000;
-      v /= 60;
-      v /= 60;
-      h = v;
-      buf.append((int) h);
-      break;
-    case HOUR_TO_MINUTE:
-      v = roundUp(v, 1000 * 60);
-      v /= 1000;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v;
-      buf.append((int) h);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      break;
-    case HOUR_TO_SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v % 60;
-      v /= 60;
-      m = v % 60;
-      v /= 60;
-      h = v;
-      buf.append((int) h);
-      buf.append(':');
-      number(buf, (int) m, 2);
-      buf.append(':');
-      number(buf, (int) s, 2);
-      fraction(buf, scale, ms);
-      break;
-    case MINUTE_TO_SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v % 60;
-      v /= 60;
-      m = v;
-      buf.append((int) m);
-      buf.append(':');
-      number(buf, (int) s, 2);
-      fraction(buf, scale, ms);
-      break;
-    case MINUTE:
-      v = roundUp(v, 1000 * 60);
-      v /= 1000;
-      v /= 60;
-      m = v;
-      buf.append((int) m);
-      break;
-    case SECOND:
-      v = roundUp(v, powerX(10, 3 - scale));
-      ms = v % 1000;
-      v /= 1000;
-      s = v;
-      buf.append((int) s);
-      fraction(buf, scale, ms);
-      break;
-    default:
-      throw new AssertionError(range);
-    }
-    return buf.toString();
-  }
-
-  /**
-   * Rounds a dividend to the nearest divisor.
-   * For example roundUp(31, 10) yields 30; roundUp(37, 10) yields 40.
-   * @param dividend Number to be divided
-   * @param divisor Number to divide by
-   * @return Rounded dividend
-   */
-  private static long roundUp(long dividend, long divisor) {
-    long remainder = dividend % divisor;
-    dividend -= remainder;
-    if (remainder * 2 > divisor) {
-      dividend += divisor;
-    }
-    return dividend;
-  }
-
-  private static int roundUp(int dividend, int divisor) {
-    int remainder = dividend % divisor;
-    dividend -= remainder;
-    if (remainder * 2 > divisor) {
-      dividend += divisor;
-    }
-    return dividend;
-  }
-
-  private static void fraction(StringBuilder buf, int scale, long ms) {
-    if (scale > 0) {
-      buf.append('.');
-      long v1 = scale == 3 ? ms
-          : scale == 2 ? ms / 10
-          : scale == 1 ? ms / 100
-            : 0;
-      number(buf, (int) v1, scale);
-    }
-  }
-
-  /** Helper for "array element reference". Caller has already ensured that
-   * array and index are not null. Index is 1-based, per SQL. */
-  public static Object arrayItem(List list, int item) {
-    if (item < 1 || item > list.size()) {
-      return null;
-    }
-    return list.get(item - 1);
-  }
-
-  /** Helper for "map element reference". Caller has already ensured that
-   * array and index are not null. Index is 1-based, per SQL. */
-  public static Object mapItem(Map map, Object item) {
-    return map.get(item);
-  }
-
-  /** Implements the {@code [ ... ]} operator on an object whose type is not
-   * known until runtime.
-   */
-  public static Object item(Object object, Object index) {
-    if (object instanceof Map) {
-      return ((Map) object).get(index);
-    }
-    if (object instanceof List && index instanceof Number) {
-      List list = (List) object;
-      return list.get(((Number) index).intValue());
-    }
-    return null;
-  }
-
-  /** NULL &rarr; FALSE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
-  public static boolean isTrue(Boolean b) {
-    return b != null && b;
-  }
-
-  /** NULL &rarr; TRUE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
-  public static boolean isNotFalse(Boolean b) {
-    return b == null || b;
-  }
-
-  /** NULL &rarr; NULL, FALSE &rarr; TRUE, TRUE &rarr; FALSE. */
-  public static Boolean not(Boolean b) {
-    return (b == null) ? null : !b;
-  }
-
-  /** Converts a JDBC array to a list. */
-  public static List arrayToList(final java.sql.Array a) {
-    if (a == null) {
-      return null;
-    }
-    try {
-      return Primitive.asList(a.getArray());
-    } catch (SQLException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /** Support the SLICE function. */
-  public static List slice(List list) {
-    return list;
-  }
-
-  /** Support the ELEMENT function. */
-  public static Object element(List list) {
-    switch (list.size()) {
-    case 0:
-      return null;
-    case 1:
-      return list.get(0);
-    default:
-      throw new RuntimeException("more than one value");
-    }
-  }
-
-  /** Returns a lambda that converts a list to an enumerable. */
-  public static <E> Function1<List<E>, Enumerable<E>> listToEnumerable() {
-    //noinspection unchecked
-    return (Function1<List<E>, Enumerable<E>>) (Function1) LIST_AS_ENUMERABLE;
-  }
-
-  /** A range of time units. The first is more significant than the
-   * other (e.g. year-to-day) or the same as the other
-   * (e.g. month). */
-  public enum TimeUnitRange {
-    YEAR,
-    YEAR_TO_MONTH,
-    MONTH,
-    DAY,
-    DAY_TO_HOUR,
-    DAY_TO_MINUTE,
-    DAY_TO_SECOND,
-    HOUR,
-    HOUR_TO_MINUTE,
-    HOUR_TO_SECOND,
-    MINUTE,
-    MINUTE_TO_SECOND,
-    SECOND;
-
-    /** Whether this is in the YEAR-TO-MONTH family of intervals. */
-    public boolean monthly() {
-      return ordinal() <= MONTH.ordinal();
-    }
-  }
-}
-
-// End SqlFunctions.java

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java b/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
new file mode 100644
index 0000000..918b4bf
--- /dev/null
+++ b/atopcalcite/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableJoin.java
@@ -0,0 +1,187 @@
+/*
+ * OVERRIDE POINT:
+ * - constructor was private instead of public 
+ */
+
+/*
+ * 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.calcite.adapter.enumerable;
+
+import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.Expression;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelNodes;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.rules.EquiJoin;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.BuiltInMethod;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.Set;
+
+/** Implementation of {@link org.apache.calcite.rel.core.Join} in
+ * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
+public class EnumerableJoin extends EquiJoin implements EnumerableRel {
+  public EnumerableJoin( // OVERRIDE POINT, the constructor was private
+      RelOptCluster cluster,
+      RelTraitSet traits,
+      RelNode left,
+      RelNode right,
+      RexNode condition,
+      ImmutableIntList leftKeys,
+      ImmutableIntList rightKeys,
+      JoinRelType joinType,
+      Set<String> variablesStopped)
+      throws InvalidRelException {
+    super(
+        cluster,
+        traits,
+        left,
+        right,
+        condition,
+        leftKeys,
+        rightKeys,
+        joinType,
+        variablesStopped);
+  }
+
+  @Override public EnumerableJoin copy(RelTraitSet traitSet, RexNode condition,
+      RelNode left, RelNode right, JoinRelType joinType,
+      boolean semiJoinDone) {
+    final JoinInfo joinInfo = JoinInfo.of(left, right, condition);
+    assert joinInfo.isEqui();
+    try {
+      return new EnumerableJoin(getCluster(), traitSet, left, right,
+          condition, joinInfo.leftKeys, joinInfo.rightKeys, joinType,
+          variablesStopped);
+    } catch (InvalidRelException e) {
+      // Semantic error not possible. Must be a bug. Convert to
+      // internal error.
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    double rowCount = RelMetadataQuery.getRowCount(this);
+
+    // Joins can be flipped, and for many algorithms, both versions are viable
+    // and have the same cost. To make the results stable between versions of
+    // the planner, make one of the versions slightly more expensive.
+    switch (joinType) {
+    case RIGHT:
+      rowCount = addEpsilon(rowCount);
+      break;
+    default:
+      if (RelNodes.COMPARATOR.compare(left, right) > 0) {
+        rowCount = addEpsilon(rowCount);
+      }
+    }
+
+    // Cheaper if the smaller number of rows is coming from the LHS.
+    // Model this by adding L log L to the cost.
+    final double rightRowCount = right.getRows();
+    final double leftRowCount = left.getRows();
+    if (Double.isInfinite(leftRowCount)) {
+      rowCount = leftRowCount;
+    } else {
+      rowCount += Util.nLogN(leftRowCount);
+    }
+    if (Double.isInfinite(rightRowCount)) {
+      rowCount = rightRowCount;
+    } else {
+      rowCount += rightRowCount;
+    }
+    return planner.getCostFactory().makeCost(rowCount, 0, 0);
+  }
+
+  private double addEpsilon(double d) {
+    assert d >= 0d;
+    final double d0 = d;
+    if (d < 10) {
+      // For small d, adding 1 would change the value significantly.
+      d *= 1.001d;
+      if (d != d0) {
+        return d;
+      }
+    }
+    // For medium d, add 1. Keeps integral values integral.
+    ++d;
+    if (d != d0) {
+      return d;
+    }
+    // For large d, adding 1 might not change the value. Add .1%.
+    // If d is NaN, this still will probably not change the value. That's OK.
+    d *= 1.001d;
+    return d;
+  }
+
+  public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
+    BlockBuilder builder = new BlockBuilder();
+    final Result leftResult =
+        implementor.visitChild(this, 0, (EnumerableRel) left, pref);
+    Expression leftExpression =
+        builder.append(
+            "left", leftResult.block);
+    final Result rightResult =
+        implementor.visitChild(this, 1, (EnumerableRel) right, pref);
+    Expression rightExpression =
+        builder.append(
+            "right", rightResult.block);
+    final PhysType physType =
+        PhysTypeImpl.of(
+            implementor.getTypeFactory(), getRowType(), pref.preferArray());
+    final PhysType keyPhysType =
+        leftResult.physType.project(
+            leftKeys, JavaRowFormat.LIST);
+    return implementor.result(
+        physType,
+        builder.append(
+            Expressions.call(
+                leftExpression,
+                BuiltInMethod.JOIN.method,
+                Expressions.list(
+                    rightExpression,
+                    leftResult.physType.generateAccessor(leftKeys),
+                    rightResult.physType.generateAccessor(rightKeys),
+                    EnumUtils.joinSelector(joinType,
+                        physType,
+                        ImmutableList.of(
+                            leftResult.physType, rightResult.physType)))
+                    .append(
+                        Util.first(keyPhysType.comparer(),
+                            Expressions.constant(null)))
+                    .append(
+                        Expressions.constant(joinType.generatesNullsOnLeft()))
+                    .append(
+                        Expressions.constant(
+                            joinType.generatesNullsOnRight())))).toBlock());
+  }
+
+}
+
+// End EnumerableJoin.java


[05/13] incubator-kylin git commit: KYLIN-780, query compile pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
index 54cafe5..9f7009e 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
@@ -20,31 +20,29 @@ package org.apache.kylin.query.relnode;
 
 import java.util.List;
 
-import net.hydromatic.linq4j.expressions.Blocks;
-import net.hydromatic.linq4j.expressions.Expressions;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.PhysType;
-import net.hydromatic.optiq.rules.java.PhysTypeImpl;
-
+import org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.calcite.linq4j.tree.Blocks;
+import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.kylin.metadata.realization.IRealization;
 import org.apache.kylin.query.routing.NoRealizationFoundException;
 import org.apache.kylin.query.routing.QueryRouter;
 import org.apache.kylin.query.schema.OLAPTable;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.convert.ConverterRelImpl;
-import org.eigenbase.relopt.ConventionTraitDef;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataType;
 
 /**
- * @author xjiang
  */
-public class OLAPToEnumerableConverter extends ConverterRelImpl implements EnumerableRel {
+public class OLAPToEnumerableConverter extends ConverterImpl implements EnumerableRel {
 
     public OLAPToEnumerableConverter(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
         super(cluster, ConventionTraitDef.INSTANCE, traits, input);
@@ -64,7 +62,7 @@ public class OLAPToEnumerableConverter extends ConverterRelImpl implements Enume
     public Result implement(EnumerableRelImplementor enumImplementor, Prefer pref) {
         // post-order travel children
         OLAPRel.OLAPImplementor olapImplementor = new OLAPRel.OLAPImplementor();
-        olapImplementor.visitChild(getChild(), this);
+        olapImplementor.visitChild(getInput(), this);
 
         // find cube from olap context
         try {
@@ -84,10 +82,10 @@ public class OLAPToEnumerableConverter extends ConverterRelImpl implements Enume
 
         // rewrite query if necessary
         OLAPRel.RewriteImplementor rewriteImplementor = new OLAPRel.RewriteImplementor();
-        rewriteImplementor.visitChild(this, getChild());
+        rewriteImplementor.visitChild(this, getInput());
 
         // build java implementation
-        EnumerableRel child = (EnumerableRel) getChild();
+        EnumerableRel child = (EnumerableRel) getInput();
         OLAPRel.JavaImplementor javaImplementor = new OLAPRel.JavaImplementor(enumImplementor);
         return javaImplementor.visitChild(this, 0, child, pref);
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/schema/OLAPSchema.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/schema/OLAPSchema.java b/query/src/main/java/org/apache/kylin/query/schema/OLAPSchema.java
index 332a5ff..27d456f 100644
--- a/query/src/main/java/org/apache/kylin/query/schema/OLAPSchema.java
+++ b/query/src/main/java/org/apache/kylin/query/schema/OLAPSchema.java
@@ -22,9 +22,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.impl.AbstractSchema;
-
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.metadata.MetadataManager;
@@ -33,7 +32,6 @@ import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.project.ProjectManager;
 
 /**
- * @author xjiang
  */
 public class OLAPSchema extends AbstractSchema {
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/schema/OLAPSchemaFactory.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/schema/OLAPSchemaFactory.java b/query/src/main/java/org/apache/kylin/query/schema/OLAPSchemaFactory.java
index 1845beb..f8711b1 100644
--- a/query/src/main/java/org/apache/kylin/query/schema/OLAPSchemaFactory.java
+++ b/query/src/main/java/org/apache/kylin/query/schema/OLAPSchemaFactory.java
@@ -25,12 +25,10 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
-import net.hydromatic.optiq.Schema;
-import net.hydromatic.optiq.SchemaFactory;
-import net.hydromatic.optiq.SchemaPlus;
-
-import org.eigenbase.util14.ConversionUtil;
-
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.util.ConversionUtil;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.metadata.model.DatabaseDesc;
 import org.apache.kylin.metadata.model.TableDesc;
@@ -38,7 +36,6 @@ import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.project.ProjectManager;
 
 /**
- * @author xjiang
  */
 public class OLAPSchemaFactory implements SchemaFactory {
 
@@ -52,7 +49,8 @@ public class OLAPSchemaFactory implements SchemaFactory {
          * object and does not require additional codec.
          * 
          * Ref SaffronProperties.defaultCharset
-         * SqlUtil.translateCharacterSetName() NlsString constructor()
+         * Ref SqlUtil.translateCharacterSetName() 
+         * Ref NlsString constructor()
          */
         System.setProperty("saffron.default.charset", ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
         System.setProperty("saffron.default.nationalcharset", ConversionUtil.NATIVE_UTF16_CHARSET_NAME);
@@ -109,13 +107,7 @@ public class OLAPSchemaFactory implements SchemaFactory {
                 out.write("            \"factory\": \"org.apache.kylin.query.schema.OLAPSchemaFactory\",\n");
                 out.write("            \"operand\": {\n");
                 out.write("                \"" + SCHEMA_PROJECT + "\": \"" + project + "\"\n");
-                out.write("            },\n");
-                out.write("           \"functions\": [\n");
-                out.write("                 {\n");
-                out.write("                     \"name\": \"QUARTER\",\n");
-                out.write("                     \"className\": \"org.apache.kylin.query.sqlfunc.QuarterFunc\"\n");
-                out.write("                 }\n");
-                out.write("            ]\n");
+                out.write("            }\n");
                 out.write("        }\n");
 
                 if (++counter != schemaCounts.size()) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
index cd04230..42b4058 100644
--- a/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
+++ b/query/src/main/java/org/apache/kylin/query/schema/OLAPTable.java
@@ -19,33 +19,30 @@
 package org.apache.kylin.query.schema;
 
 import java.util.ArrayList;
-import java.util.BitSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
-import net.hydromatic.linq4j.Enumerable;
-import net.hydromatic.linq4j.Enumerator;
-import net.hydromatic.linq4j.QueryProvider;
-import net.hydromatic.linq4j.Queryable;
-import net.hydromatic.optiq.DataContext;
-import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Statistic;
-import net.hydromatic.optiq.Statistics;
-import net.hydromatic.optiq.TranslatableTable;
-import net.hydromatic.optiq.impl.AbstractTableQueryable;
-import net.hydromatic.optiq.impl.java.AbstractQueryableTable;
-
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelOptTable.ToRelContext;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.sql.type.SqlTypeUtil;
-
-import com.google.common.collect.Lists;
+import org.apache.calcite.DataContext;
+import org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Enumerator;
+import org.apache.calcite.linq4j.QueryProvider;
+import org.apache.calcite.linq4j.Queryable;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptTable.ToRelContext;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Statistics;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTableQueryable;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.MeasureDesc;
@@ -55,10 +52,9 @@ import org.apache.kylin.query.enumerator.OLAPQuery;
 import org.apache.kylin.query.enumerator.OLAPQuery.EnumeratorTypeEnum;
 import org.apache.kylin.query.relnode.OLAPTableScan;
 
+import com.google.common.collect.Lists;
+
 /**
- * 
- * @author xjiang
- * 
  */
 public class OLAPTable extends AbstractQueryableTable implements TranslatableTable {
 
@@ -215,7 +211,7 @@ public class OLAPTable extends AbstractQueryableTable implements TranslatableTab
 
     @Override
     public Statistic getStatistic() {
-        List<BitSet> keys = new ArrayList<BitSet>();
+        List<ImmutableBitSet> keys = new ArrayList<ImmutableBitSet>();
         return Statistics.of(100, keys);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterBase.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterBase.java b/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterBase.java
deleted file mode 100644
index 1bd0dff..0000000
--- a/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterBase.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.kylin.query.sqlfunc;
-
-import net.hydromatic.optiq.runtime.SqlFunctions;
-import net.hydromatic.optiq.runtime.SqlFunctions.TimeUnitRange;
-
-/**
- * @author xjiang
- * 
- */
-public abstract class QuarterBase {
-
-    /**
-     * According to jvm spec, it return self method before parent.
-     * So, we keep Date in parent and int in child
-     */
-    public static long eval(int date) {
-        long month = SqlFunctions.unixDateExtract(TimeUnitRange.MONTH, date);
-        return month / 4 + 1;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterFunc.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterFunc.java b/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterFunc.java
deleted file mode 100644
index 179caf1..0000000
--- a/query/src/main/java/org/apache/kylin/query/sqlfunc/QuarterFunc.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.kylin.query.sqlfunc;
-
-import java.sql.Date;
-
-/**
- * @author xjiang
- * 
- */
-public class QuarterFunc extends QuarterBase {
-    private QuarterFunc() {
-    }
-
-    public static long eval(Date date) {
-        throw new UnsupportedOperationException();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
index 027202d..09499ee 100644
--- a/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
+++ b/server/src/main/java/org/apache/kylin/rest/service/QueryService.java
@@ -41,14 +41,19 @@ import java.util.Set;
 
 import javax.sql.DataSource;
 
-import net.hydromatic.avatica.ColumnMetaData.Rep;
-
+import org.apache.calcite.avatica.ColumnMetaData.Rep;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.persistence.HBaseConnection;
 import org.apache.kylin.common.util.Bytes;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.query.relnode.OLAPContext;
 import org.apache.kylin.rest.constant.Constant;
 import org.apache.kylin.rest.metrics.QueryMetrics;
 import org.apache.kylin.rest.model.ColumnMeta;
@@ -67,13 +72,6 @@ import org.springframework.security.access.prepost.PreAuthorize;
 import org.springframework.security.core.context.SecurityContextHolder;
 import org.springframework.stereotype.Component;
 
-import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.persistence.HBaseConnection;
-import org.apache.kylin.cube.CubeInstance;
-import org.apache.kylin.cube.CubeManager;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.query.relnode.OLAPContext;
-
 /**
  * @author xduo
  */


[11/13] incubator-kylin git commit: KYLIN-780 minor changes about debug

Posted by li...@apache.org.
KYLIN-780 minor changes about debug


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/621a80c9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/621a80c9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/621a80c9

Branch: refs/heads/KYLIN-780
Commit: 621a80c9457c6038e711981227deb76936830939
Parents: 5d94f1a
Author: Li, Yang <ya...@ebay.com>
Authored: Wed Jun 17 18:13:44 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Thu Jun 18 15:20:08 2015 +0800

----------------------------------------------------------------------
 .../java/org/apache/kylin/metadata/project/ProjectL2Cache.java | 2 +-
 .../java/org/apache/kylin/query/enumerator/CubeEnumerator.java | 5 +----
 .../java/org/apache/kylin/query/relnode/OLAPAggregateRel.java  | 2 +-
 .../main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java  | 1 -
 .../src/main/java/org/apache/kylin/query/relnode/OLAPRel.java  | 5 +++++
 .../apache/kylin/query/relnode/OLAPToEnumerableConverter.java  | 6 ++++++
 .../test/java/org/apache/kylin/query/test/KylinTestBase.java   | 2 +-
 7 files changed, 15 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
----------------------------------------------------------------------
diff --git a/metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java b/metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
index 10c4472..bc1b3a6 100644
--- a/metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
+++ b/metadata/src/main/java/org/apache/kylin/metadata/project/ProjectL2Cache.java
@@ -256,7 +256,7 @@ class ProjectL2Cache {
     private static class TableCache {
         private boolean exposed = false;
         private TableDesc tableDesc;
-        private Set<ColumnDesc> exposedColumns = Sets.newHashSet();
+        private Set<ColumnDesc> exposedColumns = Sets.newLinkedHashSet();
         private Set<IRealization> realizations = Sets.newHashSet();
 
         TableCache(TableDesc tableDesc) {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
index cc89119..b83e449 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/CubeEnumerator.java
@@ -132,11 +132,9 @@ public class CubeEnumerator implements Enumerator<Object[]> {
         // bind dynamic variables
         bindVariable(olapContext.filter);
 
-
-
         // query storage engine
         IStorageEngine storageEngine = StorageEngineFactory.getStorageEngine(olapContext.realization);
-        ITupleIterator iterator = storageEngine.search(olapContext.storageContext,olapContext.getSQLDigest());
+        ITupleIterator iterator = storageEngine.search(olapContext.storageContext, olapContext.getSQLDigest());
         if (logger.isDebugEnabled()) {
             logger.debug("return TupleIterator...");
         }
@@ -145,7 +143,6 @@ public class CubeEnumerator implements Enumerator<Object[]> {
         return iterator;
     }
 
-
     private void bindVariable(TupleFilter filter) {
         if (filter == null) {
             return;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 24d9a3e..4581561 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -346,7 +346,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel, EnumerableRe
         try {
             enumAggRel = new EnumerableAggregate(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getInput(), false, this.groupSet, this.groupSets, rewriteAggCalls);
         } catch (InvalidRelException e) {
-            throw new IllegalStateException("Can't create EnumerableAggregateRel!", e);
+            throw new IllegalStateException("Can't create EnumerableAggregate!", e);
         }
 
         return enumAggRel.implement(implementor, pref);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index cb0c65f..6be2b7f 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -230,7 +230,6 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
             result = super.implement(implementor, pref);
         } else {
             PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
-
             RelOptTable factTable = context.firstTableScan.getTable();
             result = implementor.result(physType, Blocks.toBlock(Expressions.call(factTable.getExpression(OLAPTable.class), "executeIndexQuery", implementor.getRootExpression(), Expressions.constant(context.id))));
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
index 1f6057f..806323c 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
@@ -148,6 +148,11 @@ public interface OLAPRel extends RelNode {
             if (parent instanceof OLAPRel) {
                 OLAPRel olapRel = (OLAPRel) parent;
                 this.parentContext = olapRel.getContext();
+                System.out.println("Parent: " + olapRel);
+            }
+            if (child instanceof OLAPRel) {
+                OLAPRel olapRel = (OLAPRel) child;
+                System.out.println(this.parentContext + " - " + olapRel.getContext() + " - " + olapRel);
             }
             return super.visitChild(parent, ordinal, child, prefer);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
index 9f7009e..57b8ee7 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPToEnumerableConverter.java
@@ -84,6 +84,12 @@ public class OLAPToEnumerableConverter extends ConverterImpl implements Enumerab
         OLAPRel.RewriteImplementor rewriteImplementor = new OLAPRel.RewriteImplementor();
         rewriteImplementor.visitChild(this, getInput());
 
+//        if (System.getProperty("calcite.debug") != null) {
+//            String dumpPlan = RelOptUtil.dumpPlan("", this, false, SqlExplainLevel.DIGEST_ATTRIBUTES);
+//            System.out.println("EXECUTION PLAN AFTER REWRITE");
+//            System.out.println(dumpPlan);
+//        }
+
         // build java implementation
         EnumerableRel child = (EnumerableRel) getInput();
         OLAPRel.JavaImplementor javaImplementor = new OLAPRel.JavaImplementor(enumImplementor);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/621a80c9/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
----------------------------------------------------------------------
diff --git a/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java b/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
index 55699d3..819d2f9 100644
--- a/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
+++ b/query/src/test/java/org/apache/kylin/query/test/KylinTestBase.java
@@ -423,7 +423,7 @@ public class KylinTestBase {
         int count = executeQuery(sql, true);
 
         if (debug) {
-            System.clearProperty("optiq.debug");
+            System.clearProperty("calcite.debug");
         }
         return count;
     }


[09/13] incubator-kylin git commit: KYLIN-780, query compile pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
new file mode 100644
index 0000000..3b1a2e0
--- /dev/null
+++ b/atopcalcite/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -0,0 +1,1261 @@
+/*
+ * OVERRIDE POINT:
+ * - divide(BigDecimal,BigDecimal), was `b0.divide(b1)`, now `b0.divide(b1, MathContext.DECIMAL64);` 
+ */
+
+/*
+ * 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.calcite.runtime;
+
+import org.apache.calcite.DataContext;
+import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.linq4j.Enumerable;
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.function.Deterministic;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.NonDeterministic;
+import org.apache.calcite.linq4j.tree.Primitive;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.MathContext;
+import java.sql.SQLException;
+import java.sql.Timestamp;
+import java.text.DecimalFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.regex.Pattern;
+
+/**
+ * Helper methods to implement SQL functions in generated code.
+ *
+ * <p>Not present: and, or, not (builtin operators are better, because they
+ * use lazy evaluation. Implementations do not check for null values; the
+ * calling code must do that.</p>
+ *
+ * <p>Many of the functions do not check for null values. This is intentional.
+ * If null arguments are possible, the code-generation framework checks for
+ * nulls before calling the functions.</p>
+ */
+@SuppressWarnings("UnnecessaryUnboxing")
+@Deterministic
+public class SqlFunctions {
+  private static final DecimalFormat DOUBLE_FORMAT =
+      new DecimalFormat("0.0E0");
+
+  private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
+
+  private static final Function1<List<Object>, Enumerable<Object>>
+  LIST_AS_ENUMERABLE =
+      new Function1<List<Object>, Enumerable<Object>>() {
+        public Enumerable<Object> apply(List<Object> list) {
+          return Linq4j.asEnumerable(list);
+        }
+      };
+
+  /** Holds, for each thread, a map from sequence name to sequence current
+   * value.
+   *
+   * <p>This is a straw man of an implementation whose main goal is to prove
+   * that sequences can be parsed, validated and planned. A real application
+   * will want persistent values for sequences, shared among threads. */
+  private static final ThreadLocal<Map<String, AtomicLong>> THREAD_SEQUENCES =
+      new ThreadLocal<Map<String, AtomicLong>>() {
+        @Override protected Map<String, AtomicLong> initialValue() {
+          return new HashMap<String, AtomicLong>();
+        }
+      };
+
+  private SqlFunctions() {
+  }
+
+  /** SQL SUBSTRING(string FROM ... FOR ...) function. */
+  public static String substring(String s, int from, int for_) {
+    return s.substring(from - 1, Math.min(from - 1 + for_, s.length()));
+  }
+
+  /** SQL SUBSTRING(string FROM ...) function. */
+  public static String substring(String s, int from) {
+    return s.substring(from - 1);
+  }
+
+  /** SQL UPPER(string) function. */
+  public static String upper(String s) {
+    return s.toUpperCase();
+  }
+
+  /** SQL LOWER(string) function. */
+  public static String lower(String s) {
+    return s.toLowerCase();
+  }
+
+  /** SQL INITCAP(string) function. */
+  public static String initcap(String s) {
+    // Assumes Alpha as [A-Za-z0-9]
+    // white space is treated as everything else.
+    final int len = s.length();
+    boolean start = true;
+    final StringBuilder newS = new StringBuilder();
+
+    for (int i = 0; i < len; i++) {
+      char curCh = s.charAt(i);
+      final int c = (int) curCh;
+      if (start) {  // curCh is whitespace or first character of word.
+        if (c > 47 && c < 58) { // 0-9
+          start = false;
+        } else if (c > 64 && c < 91) {  // A-Z
+          start = false;
+        } else if (c > 96 && c < 123) {  // a-z
+          start = false;
+          curCh = (char) (c - 32); // Uppercase this character
+        }
+        // else {} whitespace
+      } else {  // Inside of a word or white space after end of word.
+        if (c > 47 && c < 58) { // 0-9
+          // noop
+        } else if (c > 64 && c < 91) {  // A-Z
+          curCh = (char) (c + 32); // Lowercase this character
+        } else if (c > 96 && c < 123) {  // a-z
+          // noop
+        } else { // whitespace
+          start = true;
+        }
+      }
+      newS.append(curCh);
+    } // for each character in s
+    return newS.toString();
+  }
+
+  /** SQL CHARACTER_LENGTH(string) function. */
+  public static int charLength(String s) {
+    return s.length();
+  }
+
+  /** SQL {@code string || string} operator. */
+  public static String concat(String s0, String s1) {
+    return s0 + s1;
+  }
+
+  /** SQL {@code binary || binary} operator. */
+  public static ByteString concat(ByteString s0, ByteString s1) {
+    return s0.concat(s1);
+  }
+
+  /** SQL {@code RTRIM} function applied to string. */
+  public static String rtrim(String s) {
+    return trim_(s, false, true, ' ');
+  }
+
+  /** SQL {@code LTRIM} function. */
+  public static String ltrim(String s) {
+    return trim_(s, true, false, ' ');
+  }
+
+  /** SQL {@code TRIM(... seek FROM s)} function. */
+  public static String trim(boolean leading, boolean trailing, String seek,
+      String s) {
+    return trim_(s, leading, trailing, seek.charAt(0));
+  }
+
+  /** SQL {@code TRIM} function. */
+  private static String trim_(String s, boolean left, boolean right, char c) {
+    int j = s.length();
+    if (right) {
+      for (;;) {
+        if (j == 0) {
+          return "";
+        }
+        if (s.charAt(j - 1) != c) {
+          break;
+        }
+        --j;
+      }
+    }
+    int i = 0;
+    if (left) {
+      for (;;) {
+        if (i == j) {
+          return "";
+        }
+        if (s.charAt(i) != c) {
+          break;
+        }
+        ++i;
+      }
+    }
+    return s.substring(i, j);
+  }
+
+  /** SQL {@code TRIM} function applied to binary string. */
+  public static ByteString trim(ByteString s) {
+    return trim_(s, true, true);
+  }
+
+  /** Helper for CAST. */
+  public static ByteString rtrim(ByteString s) {
+    return trim_(s, false, true);
+  }
+
+  /** SQL {@code TRIM} function applied to binary string. */
+  private static ByteString trim_(ByteString s, boolean left, boolean right) {
+    int j = s.length();
+    if (right) {
+      for (;;) {
+        if (j == 0) {
+          return ByteString.EMPTY;
+        }
+        if (s.byteAt(j - 1) != 0) {
+          break;
+        }
+        --j;
+      }
+    }
+    int i = 0;
+    if (left) {
+      for (;;) {
+        if (i == j) {
+          return ByteString.EMPTY;
+        }
+        if (s.byteAt(i) != 0) {
+          break;
+        }
+        ++i;
+      }
+    }
+    return s.substring(i, j);
+  }
+
+  /** SQL {@code OVERLAY} function. */
+  public static String overlay(String s, String r, int start) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+        + r
+        + s.substring(start - 1 + r.length());
+  }
+
+  /** SQL {@code OVERLAY} function. */
+  public static String overlay(String s, String r, int start, int length) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+        + r
+        + s.substring(start - 1 + length);
+  }
+
+  /** SQL {@code OVERLAY} function applied to binary strings. */
+  public static ByteString overlay(ByteString s, ByteString r, int start) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+           .concat(r)
+           .concat(s.substring(start - 1 + r.length()));
+  }
+
+  /** SQL {@code OVERLAY} function applied to binary strings. */
+  public static ByteString overlay(ByteString s, ByteString r, int start,
+      int length) {
+    if (s == null || r == null) {
+      return null;
+    }
+    return s.substring(0, start - 1)
+           .concat(r)
+           .concat(s.substring(start - 1 + length));
+  }
+
+  /** SQL {@code LIKE} function. */
+  public static boolean like(String s, String pattern) {
+    final String regex = Like.sqlToRegexLike(pattern, null);
+    return Pattern.matches(regex, s);
+  }
+
+  /** SQL {@code LIKE} function with escape. */
+  public static boolean like(String s, String pattern, String escape) {
+    final String regex = Like.sqlToRegexLike(pattern, escape);
+    return Pattern.matches(regex, s);
+  }
+
+  /** SQL {@code SIMILAR} function. */
+  public static boolean similar(String s, String pattern) {
+    final String regex = Like.sqlToRegexSimilar(pattern, null);
+    return Pattern.matches(regex, s);
+  }
+
+  /** SQL {@code SIMILAR} function with escape. */
+  public static boolean similar(String s, String pattern, String escape) {
+    final String regex = Like.sqlToRegexSimilar(pattern, escape);
+    return Pattern.matches(regex, s);
+  }
+
+  // =
+
+  /** SQL = operator applied to Object values (including String; neither
+   * side may be null). */
+  public static boolean eq(Object b0, Object b1) {
+    return b0.equals(b1);
+  }
+
+  /** SQL = operator applied to BigDecimal values (neither may be null). */
+  public static boolean eq(BigDecimal b0, BigDecimal b1) {
+    return b0.stripTrailingZeros().equals(b1.stripTrailingZeros());
+  }
+
+  // <>
+
+  /** SQL &lt;&gt; operator applied to Object values (including String;
+   * neither side may be null). */
+  public static boolean ne(Object b0, Object b1) {
+    return !b0.equals(b1);
+  }
+
+  /** SQL &lt;&gt; operator applied to BigDecimal values. */
+  public static boolean ne(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) != 0;
+  }
+
+  // <
+
+  /** SQL &lt; operator applied to boolean values. */
+  public static boolean lt(boolean b0, boolean b1) {
+    return compare(b0, b1) < 0;
+  }
+
+  /** SQL &lt; operator applied to String values. */
+  public static boolean lt(String b0, String b1) {
+    return b0.compareTo(b1) < 0;
+  }
+
+  /** SQL &lt; operator applied to ByteString values. */
+  public static boolean lt(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) < 0;
+  }
+
+  /** SQL &lt; operator applied to BigDecimal values. */
+  public static boolean lt(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) < 0;
+  }
+
+  // <=
+
+  /** SQL &le; operator applied to boolean values. */
+  public static boolean le(boolean b0, boolean b1) {
+    return compare(b0, b1) <= 0;
+  }
+
+  /** SQL &le; operator applied to String values. */
+  public static boolean le(String b0, String b1) {
+    return b0.compareTo(b1) <= 0;
+  }
+
+  /** SQL &le; operator applied to ByteString values. */
+  public static boolean le(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) <= 0;
+  }
+
+  /** SQL &le; operator applied to BigDecimal values. */
+  public static boolean le(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) <= 0;
+  }
+
+  // >
+
+  /** SQL &gt; operator applied to boolean values. */
+  public static boolean gt(boolean b0, boolean b1) {
+    return compare(b0, b1) > 0;
+  }
+
+  /** SQL &gt; operator applied to String values. */
+  public static boolean gt(String b0, String b1) {
+    return b0.compareTo(b1) > 0;
+  }
+
+  /** SQL &gt; operator applied to ByteString values. */
+  public static boolean gt(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) > 0;
+  }
+
+  /** SQL &gt; operator applied to BigDecimal values. */
+  public static boolean gt(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) > 0;
+  }
+
+  // >=
+
+  /** SQL &ge; operator applied to boolean values. */
+  public static boolean ge(boolean b0, boolean b1) {
+    return compare(b0, b1) >= 0;
+  }
+
+  /** SQL &ge; operator applied to String values. */
+  public static boolean ge(String b0, String b1) {
+    return b0.compareTo(b1) >= 0;
+  }
+
+  /** SQL &ge; operator applied to ByteString values. */
+  public static boolean ge(ByteString b0, ByteString b1) {
+    return b0.compareTo(b1) >= 0;
+  }
+
+  /** SQL &ge; operator applied to BigDecimal values. */
+  public static boolean ge(BigDecimal b0, BigDecimal b1) {
+    return b0.compareTo(b1) >= 0;
+  }
+
+  // +
+
+  /** SQL <code>+</code> operator applied to int values. */
+  public static int plus(int b0, int b1) {
+    return b0 + b1;
+  }
+
+  /** SQL <code>+</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer plus(Integer b0, int b1) {
+    return b0 == null ? null : (b0 + b1);
+  }
+
+  /** SQL <code>+</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer plus(int b0, Integer b1) {
+    return b1 == null ? null : (b0 + b1);
+  }
+
+  /** SQL <code>+</code> operator applied to nullable int values. */
+  public static Integer plus(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 + b1);
+  }
+
+  /** SQL <code>+</code> operator applied to nullable long and int values. */
+  public static Long plus(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() + b1.longValue());
+  }
+
+  /** SQL <code>+</code> operator applied to nullable int and long values. */
+  public static Long plus(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() + b1.longValue());
+  }
+
+  /** SQL <code>+</code> operator applied to BigDecimal values. */
+  public static BigDecimal plus(BigDecimal b0, BigDecimal b1) {
+    return (b0 == null || b1 == null) ? null : b0.add(b1);
+  }
+
+  // -
+
+  /** SQL <code>-</code> operator applied to int values. */
+  public static int minus(int b0, int b1) {
+    return b0 - b1;
+  }
+
+  /** SQL <code>-</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer minus(Integer b0, int b1) {
+    return b0 == null ? null : (b0 - b1);
+  }
+
+  /** SQL <code>-</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer minus(int b0, Integer b1) {
+    return b1 == null ? null : (b0 - b1);
+  }
+
+  /** SQL <code>-</code> operator applied to nullable int values. */
+  public static Integer minus(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 - b1);
+  }
+
+  /** SQL <code>-</code> operator applied to nullable long and int values. */
+  public static Long minus(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() - b1.longValue());
+  }
+
+  /** SQL <code>-</code> operator applied to nullable int and long values. */
+  public static Long minus(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() - b1.longValue());
+  }
+
+  /** SQL <code>-</code> operator applied to BigDecimal values. */
+  public static BigDecimal minus(BigDecimal b0, BigDecimal b1) {
+    return (b0 == null || b1 == null) ? null : b0.subtract(b1);
+  }
+
+  // /
+
+  /** SQL <code>/</code> operator applied to int values. */
+  public static int divide(int b0, int b1) {
+    return b0 / b1;
+  }
+
+  /** SQL <code>/</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer divide(Integer b0, int b1) {
+    return b0 == null ? null : (b0 / b1);
+  }
+
+  /** SQL <code>/</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer divide(int b0, Integer b1) {
+    return b1 == null ? null : (b0 / b1);
+  }
+
+  /** SQL <code>/</code> operator applied to nullable int values. */
+  public static Integer divide(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 / b1);
+  }
+
+  /** SQL <code>/</code> operator applied to nullable long and int values. */
+  public static Long divide(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() / b1.longValue());
+  }
+
+  /** SQL <code>/</code> operator applied to nullable int and long values. */
+  public static Long divide(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() / b1.longValue());
+  }
+
+  /** SQL <code>/</code> operator applied to BigDecimal values. */
+  public static BigDecimal divide(BigDecimal b0, BigDecimal b1) {
+    // OVERRIDE POINT
+    return (b0 == null || b1 == null) ? null : b0.divide(b1, MathContext.DECIMAL64);
+  }
+
+  // *
+
+  /** SQL <code>*</code> operator applied to int values. */
+  public static int multiply(int b0, int b1) {
+    return b0 * b1;
+  }
+
+  /** SQL <code>*</code> operator applied to int values; left side may be
+   * null. */
+  public static Integer multiply(Integer b0, int b1) {
+    return b0 == null ? null : (b0 * b1);
+  }
+
+  /** SQL <code>*</code> operator applied to int values; right side may be
+   * null. */
+  public static Integer multiply(int b0, Integer b1) {
+    return b1 == null ? null : (b0 * b1);
+  }
+
+  /** SQL <code>*</code> operator applied to nullable int values. */
+  public static Integer multiply(Integer b0, Integer b1) {
+    return (b0 == null || b1 == null) ? null : (b0 * b1);
+  }
+
+  /** SQL <code>*</code> operator applied to nullable long and int values. */
+  public static Long multiply(Long b0, Integer b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() * b1.longValue());
+  }
+
+  /** SQL <code>*</code> operator applied to nullable int and long values. */
+  public static Long multiply(Integer b0, Long b1) {
+    return (b0 == null || b1 == null)
+        ? null
+        : (b0.longValue() * b1.longValue());
+  }
+
+  /** SQL <code>*</code> operator applied to BigDecimal values. */
+  public static BigDecimal multiply(BigDecimal b0, BigDecimal b1) {
+    return (b0 == null || b1 == null) ? null : b0.multiply(b1);
+  }
+
+  // EXP
+
+  /** SQL <code>EXP</code> operator applied to double values. */
+  public static double exp(double b0) {
+    return Math.exp(b0);
+  }
+
+  public static double exp(long b0) {
+    return Math.exp(b0);
+  }
+
+  // POWER
+
+  /** SQL <code>POWER</code> operator applied to double values. */
+  public static double power(double b0, double b1) {
+    return Math.pow(b0, b1);
+  }
+
+  public static double power(long b0, long b1) {
+    return Math.pow(b0, b1);
+  }
+
+  public static double power(long b0, BigDecimal b1) {
+    return Math.pow(b0, b1.doubleValue());
+  }
+
+  // LN
+
+  /** SQL {@code LN(number)} function applied to double values. */
+  public static double ln(double d) {
+    return Math.log(d);
+  }
+
+  /** SQL {@code LN(number)} function applied to long values. */
+  public static double ln(long b0) {
+    return Math.log(b0);
+  }
+
+  /** SQL {@code LN(number)} function applied to BigDecimal values. */
+  public static double ln(BigDecimal d) {
+    return Math.log(d.doubleValue());
+  }
+
+  // LOG10
+
+  /** SQL <code>LOG10(numeric)</code> operator applied to double values. */
+  public static double log10(double b0) {
+    return Math.log10(b0);
+  }
+
+  /** SQL {@code LOG10(number)} function applied to long values. */
+  public static double log10(long b0) {
+    return Math.log10(b0);
+  }
+
+  /** SQL {@code LOG10(number)} function applied to BigDecimal values. */
+  public static double log10(BigDecimal d) {
+    return Math.log10(d.doubleValue());
+  }
+
+  // MOD
+
+  /** SQL <code>MOD</code> operator applied to byte values. */
+  public static byte mod(byte b0, byte b1) {
+    return (byte) (b0 % b1);
+  }
+
+  /** SQL <code>MOD</code> operator applied to short values. */
+  public static short mod(short b0, short b1) {
+    return (short) (b0 % b1);
+  }
+
+  /** SQL <code>MOD</code> operator applied to int values. */
+  public static int mod(int b0, int b1) {
+    return b0 % b1;
+  }
+
+  /** SQL <code>MOD</code> operator applied to long values. */
+  public static long mod(long b0, long b1) {
+    return b0 % b1;
+  }
+
+  // temporary
+  public static BigDecimal mod(BigDecimal b0, int b1) {
+    return mod(b0, BigDecimal.valueOf(b1));
+  }
+
+  // temporary
+  public static int mod(int b0, BigDecimal b1) {
+    return mod(b0, b1.intValue());
+  }
+
+  public static BigDecimal mod(BigDecimal b0, BigDecimal b1) {
+    final BigDecimal[] bigDecimals = b0.divideAndRemainder(b1);
+    return bigDecimals[1];
+  }
+
+  // ABS
+
+  /** SQL <code>ABS</code> operator applied to byte values. */
+  public static byte abs(byte b0) {
+    return (byte) Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to short values. */
+  public static short abs(short b0) {
+    return (short) Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to int values. */
+  public static int abs(int b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to long values. */
+  public static long abs(long b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to float values. */
+  public static float abs(float b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to double values. */
+  public static double abs(double b0) {
+    return Math.abs(b0);
+  }
+
+  /** SQL <code>ABS</code> operator applied to BigDecimal values. */
+  public static BigDecimal abs(BigDecimal b0) {
+    return b0.abs();
+  }
+
+  // Helpers
+
+  /** Helper for implementing MIN. Somewhat similar to LEAST operator. */
+  public static <T extends Comparable<T>> T lesser(T b0, T b1) {
+    return b0 == null || b0.compareTo(b1) > 0 ? b1 : b0;
+  }
+
+  /** LEAST operator. */
+  public static <T extends Comparable<T>> T least(T b0, T b1) {
+    return b0 == null || b1 != null && b0.compareTo(b1) > 0 ? b1 : b0;
+  }
+
+  public static boolean greater(boolean b0, boolean b1) {
+    return b0 || b1;
+  }
+
+  public static boolean lesser(boolean b0, boolean b1) {
+    return b0 && b1;
+  }
+
+  public static byte greater(byte b0, byte b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static byte lesser(byte b0, byte b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static char greater(char b0, char b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static char lesser(char b0, char b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static short greater(short b0, short b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static short lesser(short b0, short b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static int greater(int b0, int b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static int lesser(int b0, int b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static long greater(long b0, long b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static long lesser(long b0, long b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static float greater(float b0, float b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static float lesser(float b0, float b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  public static double greater(double b0, double b1) {
+    return b0 > b1 ? b0 : b1;
+  }
+
+  public static double lesser(double b0, double b1) {
+    return b0 > b1 ? b1 : b0;
+  }
+
+  /** Helper for implementing MAX. Somewhat similar to GREATEST operator. */
+  public static <T extends Comparable<T>> T greater(T b0, T b1) {
+    return b0 == null || b0.compareTo(b1) < 0 ? b1 : b0;
+  }
+
+  /** GREATEST operator. */
+  public static <T extends Comparable<T>> T greatest(T b0, T b1) {
+    return b0 == null || b1 != null && b0.compareTo(b1) < 0 ? b1 : b0;
+  }
+
+  /** Boolean comparison. */
+  public static int compare(boolean x, boolean y) {
+    return x == y ? 0 : x ? 1 : -1;
+  }
+
+  /** CAST(FLOAT AS VARCHAR). */
+  public static String toString(float x) {
+    if (x == 0) {
+      return "0E0";
+    }
+    BigDecimal bigDecimal =
+        new BigDecimal(x, MathContext.DECIMAL32).stripTrailingZeros();
+    final String s = bigDecimal.toString();
+    return s.replaceAll("0*E", "E").replace("E+", "E");
+  }
+
+  /** CAST(DOUBLE AS VARCHAR). */
+  public static String toString(double x) {
+    if (x == 0) {
+      return "0E0";
+    }
+    BigDecimal bigDecimal =
+        new BigDecimal(x, MathContext.DECIMAL64).stripTrailingZeros();
+    final String s = bigDecimal.toString();
+    return s.replaceAll("0*E", "E").replace("E+", "E");
+  }
+
+  /** CAST(DECIMAL AS VARCHAR). */
+  public static String toString(BigDecimal x) {
+    final String s = x.toString();
+    if (s.startsWith("0")) {
+      // we want ".1" not "0.1"
+      return s.substring(1);
+    } else if (s.startsWith("-0")) {
+      // we want "-.1" not "-0.1"
+      return "-" + s.substring(2);
+    } else {
+      return s;
+    }
+  }
+
+  /** CAST(BOOLEAN AS VARCHAR). */
+  public static String toString(boolean x) {
+    // Boolean.toString returns lower case -- no good.
+    return x ? "TRUE" : "FALSE";
+  }
+
+  @NonDeterministic
+  private static Object cannotConvert(Object o, Class toType) {
+    throw new RuntimeException("Cannot convert " + o + " to " + toType);
+  }
+
+  /** CAST(VARCHAR AS BOOLEAN). */
+  public static boolean toBoolean(String s) {
+    s = trim_(s, true, true, ' ');
+    if (s.equalsIgnoreCase("TRUE")) {
+      return true;
+    } else if (s.equalsIgnoreCase("FALSE")) {
+      return false;
+    } else {
+      throw new RuntimeException("Invalid character for cast");
+    }
+  }
+
+  public static boolean toBoolean(Number number) {
+    return !number.equals(0);
+  }
+
+  public static boolean toBoolean(Object o) {
+    return o instanceof Boolean ? (Boolean) o
+        : o instanceof Number ? toBoolean((Number) o)
+        : o instanceof String ? toBoolean((String) o)
+        : (Boolean) cannotConvert(o, boolean.class);
+  }
+
+  // Don't need parseByte etc. - Byte.parseByte is sufficient.
+
+  public static byte toByte(Object o) {
+    return o instanceof Byte ? (Byte) o
+        : o instanceof Number ? toByte((Number) o)
+        : Byte.parseByte(o.toString());
+  }
+
+  public static byte toByte(Number number) {
+    return number.byteValue();
+  }
+
+  public static char toChar(String s) {
+    return s.charAt(0);
+  }
+
+  public static Character toCharBoxed(String s) {
+    return s.charAt(0);
+  }
+
+  public static short toShort(String s) {
+    return Short.parseShort(s.trim());
+  }
+
+  public static short toShort(Number number) {
+    return number.shortValue();
+  }
+
+  public static short toShort(Object o) {
+    return o instanceof Short ? (Short) o
+        : o instanceof Number ? toShort((Number) o)
+        : o instanceof String ? toShort((String) o)
+        : (Short) cannotConvert(o, short.class);
+  }
+
+  public static int toInt(java.util.Date v) {
+    return toInt(v, LOCAL_TZ);
+  }
+
+  public static int toInt(java.util.Date v, TimeZone timeZone) {
+    return (int) (toLong(v, timeZone)  / DateTimeUtils.MILLIS_PER_DAY);
+  }
+
+  public static Integer toIntOptional(java.util.Date v) {
+    return v == null ? null : toInt(v);
+  }
+
+  public static Integer toIntOptional(java.util.Date v, TimeZone timeZone) {
+    return v == null
+        ? null
+        : toInt(v, timeZone);
+  }
+
+  public static long toLong(Date v) {
+    return toLong(v, LOCAL_TZ);
+  }
+
+  public static int toInt(java.sql.Time v) {
+    return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
+  }
+
+  public static Integer toIntOptional(java.sql.Time v) {
+    return v == null ? null : toInt(v);
+  }
+
+  public static int toInt(String s) {
+    return Integer.parseInt(s.trim());
+  }
+
+  public static int toInt(Number number) {
+    return number.intValue();
+  }
+
+  public static int toInt(Object o) {
+    return o instanceof Integer ? (Integer) o
+        : o instanceof Number ? toInt((Number) o)
+        : o instanceof String ? toInt((String) o)
+        : (Integer) cannotConvert(o, int.class);
+  }
+
+  public static long toLong(Timestamp v) {
+    return toLong(v, LOCAL_TZ);
+  }
+
+  // mainly intended for java.sql.Timestamp but works for other dates also
+  public static long toLong(java.util.Date v, TimeZone timeZone) {
+    final long time = v.getTime();
+    return time + timeZone.getOffset(time);
+  }
+
+  // mainly intended for java.sql.Timestamp but works for other dates also
+  public static Long toLongOptional(java.util.Date v) {
+    return v == null ? null : toLong(v, LOCAL_TZ);
+  }
+
+  public static Long toLongOptional(Timestamp v, TimeZone timeZone) {
+    if (v == null) {
+      return null;
+    }
+    return toLong(v, LOCAL_TZ);
+  }
+
+  public static long toLong(String s) {
+    if (s.startsWith("199") && s.contains(":")) {
+      return Timestamp.valueOf(s).getTime();
+    }
+    return Long.parseLong(s.trim());
+  }
+
+  public static long toLong(Number number) {
+    return number.longValue();
+  }
+
+  public static long toLong(Object o) {
+    return o instanceof Long ? (Long) o
+        : o instanceof Number ? toLong((Number) o)
+        : o instanceof String ? toLong((String) o)
+        : (Long) cannotConvert(o, long.class);
+  }
+
+  public static float toFloat(String s) {
+    return Float.parseFloat(s.trim());
+  }
+
+  public static float toFloat(Number number) {
+    return number.floatValue();
+  }
+
+  public static float toFloat(Object o) {
+    return o instanceof Float ? (Float) o
+        : o instanceof Number ? toFloat((Number) o)
+            : o instanceof String ? toFloat((String) o)
+                : (Float) cannotConvert(o, float.class);
+  }
+
+  public static double toDouble(String s) {
+    return Double.parseDouble(s.trim());
+  }
+
+  public static double toDouble(Number number) {
+    return number.doubleValue();
+  }
+
+  public static double toDouble(Object o) {
+    return o instanceof Double ? (Double) o
+        : o instanceof Number ? toDouble((Number) o)
+        : o instanceof String ? toDouble((String) o)
+        : (Double) cannotConvert(o, double.class);
+  }
+
+  public static BigDecimal toBigDecimal(String s) {
+    return new BigDecimal(s.trim());
+  }
+
+  public static BigDecimal toBigDecimal(Number number) {
+    // There are some values of "long" that cannot be represented as "double".
+    // Not so "int". If it isn't a long, go straight to double.
+    return number instanceof BigDecimal ? (BigDecimal) number
+        : number instanceof BigInteger ? new BigDecimal((BigInteger) number)
+        : number instanceof Long ? new BigDecimal(number.longValue())
+        : new BigDecimal(number.doubleValue());
+  }
+
+  public static BigDecimal toBigDecimal(Object o) {
+    return o instanceof Number ? toBigDecimal((Number) o)
+        : toBigDecimal(o.toString());
+  }
+
+  // Don't need shortValueOf etc. - Short.valueOf is sufficient.
+
+  /** Helper for CAST(... AS VARCHAR(maxLength)). */
+  public static String truncate(String s, int maxLength) {
+    return s == null ? null
+        : s.length() > maxLength ? s.substring(0, maxLength)
+        : s;
+  }
+
+  /** Helper for CAST(... AS VARBINARY(maxLength)). */
+  public static ByteString truncate(ByteString s, int maxLength) {
+    return s == null ? null
+        : s.length() > maxLength ? s.substring(0, maxLength)
+        : s;
+  }
+
+  /** SQL {@code POSITION(seek IN string)} function. */
+  public static int position(String seek, String s) {
+    return s.indexOf(seek) + 1;
+  }
+
+  /** SQL {@code POSITION(seek IN string)} function. */
+  public static int position(ByteString seek, ByteString s) {
+    return s.indexOf(seek) + 1;
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static long round(long v, long x) {
+    return truncate(v + x / 2, x);
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static long truncate(long v, long x) {
+    long remainder = v % x;
+    if (remainder < 0) {
+      remainder += x;
+    }
+    return v - remainder;
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static int round(int v, int x) {
+    return truncate(v + x / 2, x);
+  }
+
+  /** Helper for rounding. Truncate(12345, 1000) returns 12000. */
+  public static int truncate(int v, int x) {
+    int remainder = v % x;
+    if (remainder < 0) {
+      remainder += x;
+    }
+    return v - remainder;
+  }
+
+  /** SQL {@code CURRENT_TIMESTAMP} function. */
+  @NonDeterministic
+  public static long currentTimestamp(DataContext root) {
+    // Cast required for JDK 1.6.
+    return (Long) DataContext.Variable.CURRENT_TIMESTAMP.get(root);
+  }
+
+  /** SQL {@code CURRENT_TIME} function. */
+  @NonDeterministic
+  public static int currentTime(DataContext root) {
+    int time = (int) (currentTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
+    if (time < 0) {
+      time += DateTimeUtils.MILLIS_PER_DAY;
+    }
+    return time;
+  }
+
+  /** SQL {@code CURRENT_DATE} function. */
+  @NonDeterministic
+  public static int currentDate(DataContext root) {
+    final long timestamp = currentTimestamp(root);
+    int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
+    final int time = (int) (timestamp % DateTimeUtils.MILLIS_PER_DAY);
+    if (time < 0) {
+      --date;
+    }
+    return date;
+  }
+
+  /** SQL {@code LOCAL_TIMESTAMP} function. */
+  @NonDeterministic
+  public static long localTimestamp(DataContext root) {
+    // Cast required for JDK 1.6.
+    return (Long) DataContext.Variable.LOCAL_TIMESTAMP.get(root);
+  }
+
+  /** SQL {@code LOCAL_TIME} function. */
+  @NonDeterministic
+  public static int localTime(DataContext root) {
+    return (int) (localTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
+  }
+
+  /** Helper for "array element reference". Caller has already ensured that
+   * array and index are not null. Index is 1-based, per SQL. */
+  public static Object arrayItem(List list, int item) {
+    if (item < 1 || item > list.size()) {
+      return null;
+    }
+    return list.get(item - 1);
+  }
+
+  /** Helper for "map element reference". Caller has already ensured that
+   * array and index are not null. Index is 1-based, per SQL. */
+  public static Object mapItem(Map map, Object item) {
+    return map.get(item);
+  }
+
+  /** Implements the {@code [ ... ]} operator on an object whose type is not
+   * known until runtime.
+   */
+  public static Object item(Object object, Object index) {
+    if (object instanceof Map) {
+      return ((Map) object).get(index);
+    }
+    if (object instanceof List && index instanceof Number) {
+      List list = (List) object;
+      return list.get(((Number) index).intValue());
+    }
+    return null;
+  }
+
+  /** NULL &rarr; FALSE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
+  public static boolean isTrue(Boolean b) {
+    return b != null && b;
+  }
+
+  /** NULL &rarr; TRUE, FALSE &rarr; FALSE, TRUE &rarr; TRUE. */
+  public static boolean isNotFalse(Boolean b) {
+    return b == null || b;
+  }
+
+  /** NULL &rarr; NULL, FALSE &rarr; TRUE, TRUE &rarr; FALSE. */
+  public static Boolean not(Boolean b) {
+    return (b == null) ? null : !b;
+  }
+
+  /** Converts a JDBC array to a list. */
+  public static List arrayToList(final java.sql.Array a) {
+    if (a == null) {
+      return null;
+    }
+    try {
+      return Primitive.asList(a.getArray());
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /** Support the {@code CURRENT VALUE OF sequence} operator. */
+  @NonDeterministic
+  public static long sequenceCurrentValue(String key) {
+    return getAtomicLong(key).get();
+  }
+
+  /** Support the {@code NEXT VALUE OF sequence} operator. */
+  @NonDeterministic
+  public static long sequenceNextValue(String key) {
+    return getAtomicLong(key).incrementAndGet();
+  }
+
+  private static AtomicLong getAtomicLong(String key) {
+    final Map<String, AtomicLong> map = THREAD_SEQUENCES.get();
+    AtomicLong atomic = map.get(key);
+    if (atomic == null) {
+      atomic = new AtomicLong();
+      map.put(key, atomic);
+    }
+    return atomic;
+  }
+
+  /** Support the SLICE function. */
+  public static List slice(List list) {
+    return list;
+  }
+
+  /** Support the ELEMENT function. */
+  public static Object element(List list) {
+    switch (list.size()) {
+    case 0:
+      return null;
+    case 1:
+      return list.get(0);
+    default:
+      throw new RuntimeException("more than one value");
+    }
+  }
+
+  /** Returns a lambda that converts a list to an enumerable. */
+  public static <E> Function1<List<E>, Enumerable<E>> listToEnumerable() {
+    //noinspection unchecked
+    return (Function1<List<E>, Enumerable<E>>) (Function1) LIST_AS_ENUMERABLE;
+  }
+
+}
+
+// End SqlFunctions.java



[13/13] incubator-kylin git commit: Merge remote-tracking branch 'apache/KYLIN-780' into KYLIN-780

Posted by li...@apache.org.
Merge remote-tracking branch 'apache/KYLIN-780' into KYLIN-780


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/fcb20446
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/fcb20446
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/fcb20446

Branch: refs/heads/KYLIN-780
Commit: fcb204467c30cf00cbbee0d4ff45525c7d34fb28
Parents: a0d8c3c 2415553
Author: Li, Yang <ya...@ebay.com>
Authored: Thu Jun 18 15:30:28 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Thu Jun 18 15:30:28 2015 +0800

----------------------------------------------------------------------

----------------------------------------------------------------------



[12/13] incubator-kylin git commit: KYLIN-780 query test pass!

Posted by li...@apache.org.
KYLIN-780 query test pass!


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/a0d8c3c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/a0d8c3c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/a0d8c3c0

Branch: refs/heads/KYLIN-780
Commit: a0d8c3c04dd503376ec1043a942979d8e6900593
Parents: 621a80c
Author: Yang Li <li...@apache.org>
Authored: Thu Jun 18 07:14:30 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Thu Jun 18 15:20:13 2015 +0800

----------------------------------------------------------------------
 .../query/enumerator/LookupTableEnumerator.java |  4 ++-
 .../kylin/query/relnode/OLAPAggregateRel.java   |  1 +
 .../kylin/query/relnode/OLAPFilterRel.java      |  1 +
 .../apache/kylin/query/relnode/OLAPJoinRel.java |  9 ++++---
 .../kylin/query/relnode/OLAPLimitRel.java       |  1 +
 .../kylin/query/relnode/OLAPProjectRel.java     | 27 ++++++++++++++------
 .../org/apache/kylin/query/relnode/OLAPRel.java | 27 ++++++++++++--------
 .../apache/kylin/query/relnode/OLAPSortRel.java |  2 +-
 .../kylin/query/relnode/OLAPTableScan.java      | 25 +++++++++---------
 9 files changed, 61 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java b/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
index 9e3e81d..3ae19dd 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/LookupTableEnumerator.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.query.enumerator;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -82,7 +83,8 @@ public class LookupTableEnumerator implements Enumerator<Object[]> {
 
     @Override
     public Object[] current() {
-        return current;
+        // NOTE if without the copy, sql_lookup/query03.sql will yields messy result. Very weird coz other lookup queries are all good.
+        return Arrays.copyOf(current, current.length);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 4581561..2003bc6 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -349,6 +349,7 @@ public class OLAPAggregateRel extends Aggregate implements OLAPRel, EnumerableRe
             throw new IllegalStateException("Can't create EnumerableAggregate!", e);
         }
 
+        ((OLAPRel.JavaImplementor) implementor).putParentContextForTableScanChild(this, enumAggRel);
         return enumAggRel.implement(implementor, pref);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
index 7d16c8d..16f5772 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPFilterRel.java
@@ -309,6 +309,7 @@ public class OLAPFilterRel extends Filter implements OLAPRel, EnumerableRel {
 
         EnumerableCalc enumCalcRel = new EnumerableCalc(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getInput(), program, ImmutableList.<RelCollation> of());
 
+        ((OLAPRel.JavaImplementor) implementor).putParentContextForTableScanChild(this, enumCalcRel);
         return enumCalcRel.implement(implementor, pref);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
index 6be2b7f..9a85f12 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPJoinRel.java
@@ -25,14 +25,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import com.google.common.base.Preconditions;
-
 import org.apache.calcite.adapter.enumerable.EnumerableJoin;
 import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
 import org.apache.calcite.adapter.enumerable.PhysType;
 import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
 import org.apache.calcite.linq4j.tree.Blocks;
 import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
@@ -56,6 +55,8 @@ import org.apache.kylin.metadata.model.JoinDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.query.schema.OLAPTable;
 
+import com.google.common.base.Preconditions;
+
 /**
  */
 public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
@@ -227,11 +228,13 @@ public class OLAPJoinRel extends EnumerableJoin implements OLAPRel {
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {
         Result result = null;
         if (this.hasSubQuery) {
+            ((OLAPRel.JavaImplementor) implementor).putParentContextForTableScanChild(this, this);
             result = super.implement(implementor, pref);
         } else {
             PhysType physType = PhysTypeImpl.of(implementor.getTypeFactory(), getRowType(), pref.preferArray());
             RelOptTable factTable = context.firstTableScan.getTable();
-            result = implementor.result(physType, Blocks.toBlock(Expressions.call(factTable.getExpression(OLAPTable.class), "executeIndexQuery", implementor.getRootExpression(), Expressions.constant(context.id))));
+            MethodCallExpression exprCall = Expressions.call(factTable.getExpression(OLAPTable.class), "executeIndexQuery", implementor.getRootExpression(), Expressions.constant(context.id));
+            result = implementor.result(physType, Blocks.toBlock(exprCall));
         }
 
         return result;

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
index ac9f3b1..b496054 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPLimitRel.java
@@ -101,6 +101,7 @@ public class OLAPLimitRel extends SingleRel implements OLAPRel, EnumerableRel {
         childRel.replaceTraitSet(EnumerableConvention.INSTANCE);
 
         EnumerableLimit enumLimit = new EnumerableLimit(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), getInput(), localOffset, localFetch);
+        ((OLAPRel.JavaImplementor) implementor).putParentContextForTableScanChild(this, enumLimit);
         Result res = enumLimit.implement(implementor, pref);
 
         childRel.replaceTraitSet(CONVENTION);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
index 2de43e3..b22d291 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPProjectRel.java
@@ -121,12 +121,28 @@ public class OLAPProjectRel extends Project implements OLAPRel, EnumerableRel {
             String fieldName = columnField.getName();
             Set<TblColRef> sourceCollector = new HashSet<TblColRef>();
             TblColRef column = translateRexNode(rex, inputColumnRowType, fieldName, sourceCollector);
+            if (column == null)
+                throw new IllegalStateException("No TblColRef found in " + rex);
             columns.add(column);
             sourceColumns.add(sourceCollector);
         }
         return new ColumnRowType(columns, sourceColumns);
     }
 
+    private TblColRef translateFirstRexInputRef(RexCall call, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector) {
+        for (RexNode operand : call.getOperands()) {
+            if (operand instanceof RexInputRef) {
+                return translateRexInputRef((RexInputRef) operand, inputColumnRowType, fieldName, sourceCollector);
+            }
+            if (operand instanceof RexCall) {
+                TblColRef r = translateFirstRexInputRef((RexCall) operand, inputColumnRowType, fieldName, sourceCollector);
+                if (r != null)
+                    return r;
+            }
+        }
+        return null;
+    }
+
     private TblColRef translateRexNode(RexNode rexNode, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector) {
         TblColRef column = null;
         if (rexNode instanceof RexInputRef) {
@@ -171,16 +187,10 @@ public class OLAPProjectRel extends Project implements OLAPRel, EnumerableRel {
     private TblColRef translateRexCall(RexCall call, ColumnRowType inputColumnRowType, String fieldName, Set<TblColRef> sourceCollector) {
         SqlOperator operator = call.getOperator();
         if (operator == SqlStdOperatorTable.EXTRACT_DATE) {
-            List<RexNode> extractDateOps = call.getOperands();
-            RexCall reinterpret = (RexCall) extractDateOps.get(1);
-            List<RexNode> reinterpretOps = reinterpret.getOperands();
-            RexInputRef inputRef = (RexInputRef) reinterpretOps.get(0);
-            return translateRexInputRef(inputRef, inputColumnRowType, fieldName, sourceCollector);
+            return translateFirstRexInputRef(call, inputColumnRowType, fieldName, sourceCollector);
         } else if (operator instanceof SqlUserDefinedFunction) {
             if (operator.getName().equals("QUARTER")) {
-                List<RexNode> quaterOps = call.getOperands();
-                RexInputRef inputRef = (RexInputRef) quaterOps.get(0);
-                return translateRexInputRef(inputRef, inputColumnRowType, fieldName, sourceCollector);
+                return translateFirstRexInputRef(call, inputColumnRowType, fieldName, sourceCollector);
             }
         } else if (operator instanceof SqlCaseOperator) {
             for (RexNode operand : call.getOperands()) {
@@ -215,6 +225,7 @@ public class OLAPProjectRel extends Project implements OLAPRel, EnumerableRel {
             enumCalcRel = new EnumerableCalc(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE), child, program, ImmutableList.<RelCollation> of());
         }
 
+        ((OLAPRel.JavaImplementor) implementor).putParentContextForTableScanChild(this, enumCalcRel);
         return enumCalcRel.implement(implementor, pref);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
index 806323c..974b95c 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPRel.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.query.relnode;
 
+import java.util.IdentityHashMap;
 import java.util.LinkedHashMap;
 import java.util.Stack;
 
@@ -30,6 +31,8 @@ import org.apache.calcite.rel.RelNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Maps;
+
 /**
  */
 public interface OLAPRel extends RelNode {
@@ -133,26 +136,30 @@ public interface OLAPRel extends RelNode {
      */
     public static class JavaImplementor extends EnumerableRelImplementor {
 
-        private OLAPContext parentContext;
+        private IdentityHashMap<EnumerableRel, OLAPContext> parentContexts = Maps.newIdentityHashMap();
+        private boolean calciteDebug = System.getProperty("calcite.debug") != null;
 
         public JavaImplementor(EnumerableRelImplementor enumImplementor) {
             super(enumImplementor.getRexBuilder(), new LinkedHashMap<String, Object>());
         }
 
-        public OLAPContext getParentContext() {
-            return parentContext;
+        public void putParentContextForTableScanChild(OLAPRel olapParent, EnumerableRel enumParent) {
+            parentContexts.put(enumParent, olapParent.getContext());
         }
-
+        
         @Override
         public EnumerableRel.Result visitChild(EnumerableRel parent, int ordinal, EnumerableRel child, EnumerableRel.Prefer prefer) {
-            if (parent instanceof OLAPRel) {
-                OLAPRel olapRel = (OLAPRel) parent;
-                this.parentContext = olapRel.getContext();
-                System.out.println("Parent: " + olapRel);
+            // OLAPTableScan is shared instance when the same table appears multiple times in the tree.
+            // Its context must be set (or corrected) right before visiting.
+            if (child instanceof OLAPTableScan) {
+                OLAPContext parentContext = parentContexts.get(parent);
+                if (parentContext != null) {
+                    ((OLAPTableScan) child).overrideContext(parentContext);
+                }
             }
-            if (child instanceof OLAPRel) {
+            if (calciteDebug && child instanceof OLAPRel) {
                 OLAPRel olapRel = (OLAPRel) child;
-                System.out.println(this.parentContext + " - " + olapRel.getContext() + " - " + olapRel);
+                System.out.println(olapRel.getContext() + " - " + olapRel);
             }
             return super.visitChild(parent, ordinal, child, prefer);
         }

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
index 2591719..5048ba2 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPSortRel.java
@@ -118,7 +118,7 @@ public class OLAPSortRel extends Sort implements EnumerableRel, OLAPRel {
         childRel.replaceTraitSet(EnumerableConvention.INSTANCE);
 
         EnumerableSort enumSort = new EnumerableSort(getCluster(), getCluster().traitSetOf(EnumerableConvention.INSTANCE, collation), getInput(), collation, offset, fetch);
-
+        ((OLAPRel.JavaImplementor) implementor).putParentContextForTableScanChild(this, enumSort);
         Result res = enumSort.implement(implementor, pref);
 
         childRel.replaceTraitSet(OLAPRel.CONVENTION);

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/a0d8c3c0/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
----------------------------------------------------------------------
diff --git a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
index f149592..afb4b60 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPTableScan.java
@@ -28,6 +28,7 @@ import org.apache.calcite.adapter.enumerable.PhysType;
 import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
 import org.apache.calcite.linq4j.tree.Blocks;
 import org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
@@ -47,11 +48,6 @@ import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.kylin.query.schema.OLAPSchema;
-import org.apache.kylin.query.schema.OLAPTable;
-
-import com.google.common.base.Preconditions;
-
 import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.query.optrule.OLAPAggregateRule;
@@ -61,6 +57,10 @@ import org.apache.kylin.query.optrule.OLAPLimitRule;
 import org.apache.kylin.query.optrule.OLAPProjectRule;
 import org.apache.kylin.query.optrule.OLAPSortRule;
 import org.apache.kylin.query.optrule.OLAPToEnumerableConverterRule;
+import org.apache.kylin.query.schema.OLAPSchema;
+import org.apache.kylin.query.schema.OLAPTable;
+
+import com.google.common.base.Preconditions;
 
 /**
  */
@@ -96,6 +96,10 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
     public OLAPContext getContext() {
         return context;
     }
+    
+    void overrideContext(OLAPContext context) {
+        this.context = context;
+    }
 
     @Override
     public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
@@ -192,21 +196,16 @@ public class OLAPTableScan extends TableScan implements OLAPRel, EnumerableRel {
             throw new IllegalStateException("implementor is not JavaImplementor");
         JavaImplementor javaImplementor = (JavaImplementor) implementor;
 
-        int ctxId = this.context.id;
-        if (javaImplementor.getParentContext() != null) {
-            ctxId = javaImplementor.getParentContext().id;
-        }
-
         PhysType physType = PhysTypeImpl.of(javaImplementor.getTypeFactory(), this.rowType, pref.preferArray());
 
         String execFunction = genExecFunc();
 
-        return javaImplementor.result(physType, Blocks.toBlock(Expressions.call(table.getExpression(OLAPTable.class), execFunction, javaImplementor.getRootExpression(), Expressions.constant(ctxId))));
+        MethodCallExpression exprCall = Expressions.call(table.getExpression(OLAPTable.class), execFunction, javaImplementor.getRootExpression(), Expressions.constant(context.id));
+        return javaImplementor.result(physType, Blocks.toBlock(exprCall));
     }
 
     private String genExecFunc() {
-        // if the table to scan is not the fact table of cube, then it's a
-        // lookup table
+        // if the table to scan is not the fact table of cube, then it's a lookup table
         if (context.hasJoin == false && tableName.equalsIgnoreCase(context.realization.getFactTable()) == false) {
             return "executeLookupTableQuery";
         } else {


[07/13] incubator-kylin git commit: KYLIN-780, query compile pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
deleted file mode 100644
index dd7ee01..0000000
--- a/atopcalcite/src/main/java/org/eigenbase/sql2rel/SqlToRelConverter.java
+++ /dev/null
@@ -1,4795 +0,0 @@
-/*
- * OVERRIDE POINT:
- * - getInSubqueryThreshold(), was `20`, now `Integer.MAX_VALUE`
- * - isTrimUnusedFields(), override to false
- * - AggConverter.visit(SqlCall), skip column reading for COUNT(COL), for https://jirap.corp.ebay.com/browse/KYLIN-104
- */
-
-/*
- * 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.eigenbase.sql2rel;
-
-import java.lang.reflect.Type;
-import java.math.*;
-import java.util.*;
-import java.util.logging.*;
-
-import org.eigenbase.rel.*;
-import org.eigenbase.rel.metadata.*;
-import org.eigenbase.relopt.*;
-import org.eigenbase.reltype.*;
-import org.eigenbase.rex.*;
-import org.eigenbase.sql.*;
-import org.eigenbase.sql.fun.*;
-import org.eigenbase.sql.parser.*;
-import org.eigenbase.sql.type.*;
-import org.eigenbase.sql.util.*;
-import org.eigenbase.sql.validate.*;
-import org.eigenbase.trace.*;
-import org.eigenbase.util.*;
-import org.eigenbase.util.mapping.Mappings;
-import org.eigenbase.util14.*;
-
-import net.hydromatic.linq4j.Ord;
-
-import net.hydromatic.optiq.ModifiableTable;
-import net.hydromatic.optiq.TranslatableTable;
-import net.hydromatic.optiq.prepare.Prepare;
-import net.hydromatic.optiq.prepare.RelOptTableImpl;
-import net.hydromatic.optiq.util.BitSets;
-
-import com.google.common.base.Function;
-import com.google.common.collect.*;
-
-import static org.eigenbase.sql.SqlUtil.stripAs;
-import static org.eigenbase.util.Static.RESOURCE;
-
-/**
- * Converts a SQL parse tree (consisting of {@link org.eigenbase.sql.SqlNode}
- * objects) into a relational algebra expression (consisting of
- * {@link org.eigenbase.rel.RelNode} objects).
- *
- * <p>The public entry points are: {@link #convertQuery},
- * {@link #convertExpression(SqlNode)}.
- */
-public class SqlToRelConverter {
-  //~ Static fields/initializers ---------------------------------------------
-
-  protected static final Logger SQL2REL_LOGGER =
-      EigenbaseTrace.getSqlToRelTracer();
-
-  private static final Function<SubQuery, SqlNode> FN =
-      new Function<SubQuery, SqlNode>() {
-        public SqlNode apply(SubQuery input) {
-          return input.node;
-        }
-      };
-
-  //~ Instance fields --------------------------------------------------------
-
-  protected final SqlValidator validator;
-  protected final RexBuilder rexBuilder;
-  protected final Prepare.CatalogReader catalogReader;
-  protected final RelOptCluster cluster;
-  private DefaultValueFactory defaultValueFactory;
-  private SubqueryConverter subqueryConverter;
-  protected final List<RelNode> leaves = new ArrayList<RelNode>();
-  private final List<SqlDynamicParam> dynamicParamSqlNodes =
-      new ArrayList<SqlDynamicParam>();
-  private final SqlOperatorTable opTab;
-  private boolean shouldConvertTableAccess;
-  protected final RelDataTypeFactory typeFactory;
-  private final SqlNodeToRexConverter exprConverter;
-  private boolean decorrelationEnabled;
-  private boolean trimUnusedFields;
-  private boolean shouldCreateValuesRel;
-  private boolean isExplain;
-  private int nDynamicParamsInExplain;
-
-  /**
-   * Fields used in name resolution for correlated subqueries.
-   */
-  private final Map<String, DeferredLookup> mapCorrelToDeferred =
-      new HashMap<String, DeferredLookup>();
-  private int nextCorrel = 0;
-
-  private static final String CORREL_PREFIX = "$cor";
-
-  /**
-   * Stack of names of datasets requested by the <code>
-   * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
-   */
-  private final Stack<String> datasetStack = new Stack<String>();
-
-  /**
-   * Mapping of non-correlated subqueries that have been converted to their
-   * equivalent constants. Used to avoid re-evaluating the subquery if it's
-   * already been evaluated.
-   */
-  private final Map<SqlNode, RexNode> mapConvertedNonCorrSubqs =
-      new HashMap<SqlNode, RexNode>();
-
-  public final RelOptTable.ViewExpander viewExpander;
-
-  //~ Constructors -----------------------------------------------------------
-  /**
-   * Creates a converter.
-   *
-   * @param viewExpander    Preparing statement
-   * @param validator       Validator
-   * @param catalogReader   Schema
-   * @param planner         Planner
-   * @param rexBuilder      Rex builder
-   * @param convertletTable Expression converter
-   */
-  public SqlToRelConverter(
-      RelOptTable.ViewExpander viewExpander,
-      SqlValidator validator,
-      Prepare.CatalogReader catalogReader,
-      RelOptPlanner planner,
-      RexBuilder rexBuilder,
-      SqlRexConvertletTable convertletTable) {
-    this.viewExpander = viewExpander;
-    this.opTab =
-        (validator
-            == null) ? SqlStdOperatorTable.instance()
-            : validator.getOperatorTable();
-    this.validator = validator;
-    this.catalogReader = catalogReader;
-    this.defaultValueFactory = new NullDefaultValueFactory();
-    this.subqueryConverter = new NoOpSubqueryConverter();
-    this.rexBuilder = rexBuilder;
-    this.typeFactory = rexBuilder.getTypeFactory();
-    RelOptQuery query = new RelOptQuery(planner);
-    this.cluster = query.createCluster(typeFactory, rexBuilder);
-    this.shouldConvertTableAccess = true;
-    this.exprConverter =
-        new SqlNodeToRexConverterImpl(convertletTable);
-    decorrelationEnabled = true;
-    trimUnusedFields = false;
-    shouldCreateValuesRel = true;
-    isExplain = false;
-    nDynamicParamsInExplain = 0;
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  /**
-   * @return the RelOptCluster in use.
-   */
-  public RelOptCluster getCluster() {
-    return cluster;
-  }
-
-  /**
-   * Returns the row-expression builder.
-   */
-  public RexBuilder getRexBuilder() {
-    return rexBuilder;
-  }
-
-  /**
-   * Returns the number of dynamic parameters encountered during translation;
-   * this must only be called after {@link #convertQuery}.
-   *
-   * @return number of dynamic parameters
-   */
-  public int getDynamicParamCount() {
-    return dynamicParamSqlNodes.size();
-  }
-
-  /**
-   * Returns the type inferred for a dynamic parameter.
-   *
-   * @param index 0-based index of dynamic parameter
-   * @return inferred type, never null
-   */
-  public RelDataType getDynamicParamType(int index) {
-    SqlNode sqlNode = dynamicParamSqlNodes.get(index);
-    if (sqlNode == null) {
-      throw Util.needToImplement("dynamic param type inference");
-    }
-    return validator.getValidatedNodeType(sqlNode);
-  }
-
-  /**
-   * Returns the current count of the number of dynamic parameters in an
-   * EXPLAIN PLAN statement.
-   *
-   * @param increment if true, increment the count
-   * @return the current count before the optional increment
-   */
-  public int getDynamicParamCountInExplain(boolean increment) {
-    int retVal = nDynamicParamsInExplain;
-    if (increment) {
-      ++nDynamicParamsInExplain;
-    }
-    return retVal;
-  }
-
-  /**
-   * @return mapping of non-correlated subqueries that have been converted to
-   * the constants that they evaluate to
-   */
-  public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
-    return mapConvertedNonCorrSubqs;
-  }
-
-  /**
-   * Adds to the current map of non-correlated converted subqueries the
-   * elements from another map that contains non-correlated subqueries that
-   * have been converted by another SqlToRelConverter.
-   *
-   * @param alreadyConvertedNonCorrSubqs the other map
-   */
-  public void addConvertedNonCorrSubqs(
-      Map<SqlNode, RexNode> alreadyConvertedNonCorrSubqs) {
-    mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs);
-  }
-
-  /**
-   * Set a new DefaultValueFactory. To have any effect, this must be called
-   * before any convert method.
-   *
-   * @param factory new DefaultValueFactory
-   */
-  public void setDefaultValueFactory(DefaultValueFactory factory) {
-    defaultValueFactory = factory;
-  }
-
-  /**
-   * Sets a new SubqueryConverter. To have any effect, this must be called
-   * before any convert method.
-   *
-   * @param converter new SubqueryConverter
-   */
-  public void setSubqueryConverter(SubqueryConverter converter) {
-    subqueryConverter = converter;
-  }
-
-  /**
-   * Indicates that the current statement is part of an EXPLAIN PLAN statement
-   *
-   * @param nDynamicParams number of dynamic parameters in the statement
-   */
-  public void setIsExplain(int nDynamicParams) {
-    isExplain = true;
-    nDynamicParamsInExplain = nDynamicParams;
-  }
-
-  /**
-   * Controls whether table access references are converted to physical rels
-   * immediately. The optimizer doesn't like leaf rels to have
-   * {@link Convention#NONE}. However, if we are doing further conversion
-   * passes (e.g. {@link RelStructuredTypeFlattener}), then we may need to
-   * defer conversion. To have any effect, this must be called before any
-   * convert method.
-   *
-   * @param enabled true for immediate conversion (the default); false to
-   *                generate logical TableAccessRel instances
-   */
-  public void enableTableAccessConversion(boolean enabled) {
-    shouldConvertTableAccess = enabled;
-  }
-
-  /**
-   * Controls whether instances of {@link ValuesRel} are generated. These may
-   * not be supported by all physical implementations. To have any effect,
-   * this must be called before any convert method.
-   *
-   * @param enabled true to allow ValuesRel to be generated (the default);
-   *                false to force substitution of ProjectRel+OneRowRel instead
-   */
-  public void enableValuesRelCreation(boolean enabled) {
-    shouldCreateValuesRel = enabled;
-  }
-
-  private void checkConvertedType(SqlNode query, RelNode result) {
-    if (!query.isA(SqlKind.DML)) {
-      // Verify that conversion from SQL to relational algebra did
-      // not perturb any type information.  (We can't do this if the
-      // SQL statement is something like an INSERT which has no
-      // validator type information associated with its result,
-      // hence the namespace check above.)
-      RelDataType convertedRowType = result.getRowType();
-      if (!checkConvertedRowType(query, convertedRowType)) {
-        RelDataType validatedRowType =
-            validator.getValidatedNodeType(query);
-        validatedRowType = uniquifyFields(validatedRowType);
-        throw Util.newInternal(
-            "Conversion to relational algebra failed to preserve "
-            + "datatypes:\n"
-            + "validated type:\n"
-            + validatedRowType.getFullTypeString()
-            + "\nconverted type:\n"
-            + convertedRowType.getFullTypeString()
-            + "\nrel:\n"
-            + RelOptUtil.toString(result));
-      }
-    }
-  }
-
-  public RelNode flattenTypes(
-      RelNode rootRel,
-      boolean restructure) {
-    RelStructuredTypeFlattener typeFlattener =
-        new RelStructuredTypeFlattener(rexBuilder, createToRelContext());
-    return typeFlattener.rewrite(rootRel, restructure);
-  }
-
-  /**
-   * If subquery is correlated and decorrelation is enabled, performs
-   * decorrelation.
-   *
-   * @param query   Query
-   * @param rootRel Root relational expression
-   * @return New root relational expression after decorrelation
-   */
-  public RelNode decorrelate(SqlNode query, RelNode rootRel) {
-    if (!enableDecorrelation()) {
-      return rootRel;
-    }
-    final RelNode result = decorrelateQuery(rootRel);
-    if (result != rootRel) {
-      checkConvertedType(query, result);
-    }
-    return result;
-  }
-
-  /**
-   * Walks over a tree of relational expressions, replacing each
-   * {@link RelNode} with a 'slimmed down' relational expression that projects
-   * only the fields required by its consumer.
-   *
-   * <p>This may make things easier for the optimizer, by removing crud that
-   * would expand the search space, but is difficult for the optimizer itself
-   * to do it, because optimizer rules must preserve the number and type of
-   * fields. Hence, this transform that operates on the entire tree, similar
-   * to the {@link RelStructuredTypeFlattener type-flattening transform}.
-   *
-   * <p>Currently this functionality is disabled in farrago/luciddb; the
-   * default implementation of this method does nothing.
-   *
-   * @param rootRel Relational expression that is at the root of the tree
-   * @return Trimmed relational expression
-   */
-  public RelNode trimUnusedFields(RelNode rootRel) {
-    // Trim fields that are not used by their consumer.
-    if (isTrimUnusedFields()) {
-      final RelFieldTrimmer trimmer = newFieldTrimmer();
-      rootRel = trimmer.trim(rootRel);
-      boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
-      if (dumpPlan) {
-        SQL2REL_LOGGER.fine(
-            RelOptUtil.dumpPlan(
-                "Plan after trimming unused fields",
-                rootRel,
-                false,
-                SqlExplainLevel.EXPPLAN_ATTRIBUTES));
-      }
-    }
-    return rootRel;
-  }
-
-  /**
-   * Creates a RelFieldTrimmer.
-   *
-   * @return Field trimmer
-   */
-  protected RelFieldTrimmer newFieldTrimmer() {
-    return new RelFieldTrimmer(validator);
-  }
-
-  /**
-   * Converts an unvalidated query's parse tree into a relational expression.
-   *
-   * @param query           Query to convert
-   * @param needsValidation Whether to validate the query before converting;
-   *                        <code>false</code> if the query has already been
-   *                        validated.
-   * @param top             Whether the query is top-level, say if its result
-   *                        will become a JDBC result set; <code>false</code> if
-   *                        the query will be part of a view.
-   */
-  public RelNode convertQuery(
-      SqlNode query,
-      final boolean needsValidation,
-      final boolean top) {
-    if (needsValidation) {
-      query = validator.validate(query);
-    }
-
-    RelNode result = convertQueryRecursive(query, top, null);
-    checkConvertedType(query, result);
-
-    boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
-    if (dumpPlan) {
-      SQL2REL_LOGGER.fine(
-          RelOptUtil.dumpPlan(
-              "Plan after converting SqlNode to RelNode",
-              result,
-              false,
-              SqlExplainLevel.EXPPLAN_ATTRIBUTES));
-    }
-
-    return result;
-  }
-
-  protected boolean checkConvertedRowType(
-      SqlNode query,
-      RelDataType convertedRowType) {
-    RelDataType validatedRowType = validator.getValidatedNodeType(query);
-    validatedRowType = uniquifyFields(validatedRowType);
-
-    return RelOptUtil.equal(
-        "validated row type",
-        validatedRowType,
-        "converted row type",
-        convertedRowType,
-        false);
-  }
-
-  protected RelDataType uniquifyFields(RelDataType rowType) {
-    return validator.getTypeFactory().createStructType(
-        RelOptUtil.getFieldTypeList(rowType),
-        SqlValidatorUtil.uniquify(rowType.getFieldNames()));
-  }
-
-  /**
-   * Converts a SELECT statement's parse tree into a relational expression.
-   */
-  public RelNode convertSelect(SqlSelect select) {
-    final SqlValidatorScope selectScope = validator.getWhereScope(select);
-    final Blackboard bb = createBlackboard(selectScope, null);
-    convertSelectImpl(bb, select);
-    return bb.root;
-  }
-
-  /**
-   * Factory method for creating translation workspace.
-   */
-  protected Blackboard createBlackboard(
-      SqlValidatorScope scope,
-      Map<String, RexNode> nameToNodeMap) {
-    return new Blackboard(scope, nameToNodeMap);
-  }
-
-  /**
-   * Implementation of {@link #convertSelect(SqlSelect)}; derived class may
-   * override.
-   */
-  protected void convertSelectImpl(
-      final Blackboard bb,
-      SqlSelect select) {
-    convertFrom(
-        bb,
-        select.getFrom());
-    convertWhere(
-        bb,
-        select.getWhere());
-
-    List<SqlNode> orderExprList = new ArrayList<SqlNode>();
-    List<RelFieldCollation> collationList =
-        new ArrayList<RelFieldCollation>();
-    gatherOrderExprs(
-        bb,
-        select,
-        select.getOrderList(),
-        orderExprList,
-        collationList);
-    final RelCollation collation =
-        cluster.traitSetOf().canonize(RelCollationImpl.of(collationList));
-
-    if (validator.isAggregate(select)) {
-      convertAgg(
-          bb,
-          select,
-          orderExprList);
-    } else {
-      convertSelectList(
-          bb,
-          select,
-          orderExprList);
-    }
-
-    if (select.isDistinct()) {
-      distinctify(bb, true);
-    }
-    convertOrder(
-        select, bb, collation, orderExprList, select.getOffset(),
-        select.getFetch());
-    bb.setRoot(bb.root, true);
-  }
-
-  /**
-   * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds
-   * a relational expression to make the results unique.
-   *
-   * <p>If the SELECT clause contains duplicate expressions, adds {@link
-   * ProjectRel}s so that we are grouping on the minimal set of keys. The
-   * performance gain isn't huge, but it is difficult to detect these
-   * duplicate expressions later.
-   *
-   * @param bb               Blackboard
-   * @param checkForDupExprs Check for duplicate expressions
-   */
-  private void distinctify(
-      Blackboard bb,
-      boolean checkForDupExprs) {
-    // Look for duplicate expressions in the project.
-    // Say we have 'select x, y, x, z'.
-    // Then dups will be {[2, 0]}
-    // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]}
-    RelNode rel = bb.root;
-    if (checkForDupExprs && (rel instanceof ProjectRel)) {
-      ProjectRel project = (ProjectRel) rel;
-      final List<RexNode> projectExprs = project.getProjects();
-      List<Integer> origins = new ArrayList<Integer>();
-      int dupCount = 0;
-      for (int i = 0; i < projectExprs.size(); i++) {
-        int x = findExpr(projectExprs.get(i), projectExprs, i);
-        if (x >= 0) {
-          origins.add(x);
-          ++dupCount;
-        } else {
-          origins.add(i);
-        }
-      }
-      if (dupCount == 0) {
-        distinctify(bb, false);
-        return;
-      }
-
-      final Map<Integer, Integer> squished = Maps.newHashMap();
-      final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
-      final List<Pair<RexNode, String>> newProjects = Lists.newArrayList();
-      for (int i = 0; i < fields.size(); i++) {
-        if (origins.get(i) == i) {
-          squished.put(i, newProjects.size());
-          newProjects.add(RexInputRef.of2(i, fields));
-        }
-      }
-      rel =
-          new ProjectRel(
-              cluster,
-              rel,
-              Pair.left(newProjects),
-              Pair.right(newProjects),
-              ProjectRel.Flags.BOXED);
-
-      bb.root = rel;
-      distinctify(bb, false);
-      rel = bb.root;
-
-      // Create the expressions to reverse the mapping.
-      // Project($0, $1, $0, $2).
-      final List<Pair<RexNode, String>> undoProjects = Lists.newArrayList();
-      for (int i = 0; i < fields.size(); i++) {
-        final int origin = origins.get(i);
-        RelDataTypeField field = fields.get(i);
-        undoProjects.add(
-            Pair.of(
-                (RexNode) new RexInputRef(
-                    squished.get(origin), field.getType()),
-                field.getName()));
-      }
-
-      rel =
-          new ProjectRel(
-              cluster,
-              rel,
-              Pair.left(undoProjects),
-              Pair.right(undoProjects),
-              ProjectRel.Flags.BOXED);
-
-      bb.setRoot(
-          rel,
-          false);
-
-      return;
-    }
-
-    // Usual case: all of the expressions in the SELECT clause are
-    // different.
-    rel =
-        createAggregate(
-            bb,
-            BitSets.range(rel.getRowType().getFieldCount()),
-            ImmutableList.<AggregateCall>of());
-
-    bb.setRoot(
-        rel,
-        false);
-  }
-
-  private int findExpr(RexNode seek, List<RexNode> exprs, int count) {
-    for (int i = 0; i < count; i++) {
-      RexNode expr = exprs.get(i);
-      if (expr.toString().equals(seek.toString())) {
-        return i;
-      }
-    }
-    return -1;
-  }
-
-  /**
-   * Converts a query's ORDER BY clause, if any.
-   *
-   * @param select        Query
-   * @param bb            Blackboard
-   * @param collation     Collation list
-   * @param orderExprList Method populates this list with orderBy expressions
-   *                      not present in selectList
-   * @param offset        Expression for number of rows to discard before
-   *                      returning first row
-   * @param fetch         Expression for number of rows to fetch
-   */
-  protected void convertOrder(
-      SqlSelect select,
-      Blackboard bb,
-      RelCollation collation,
-      List<SqlNode> orderExprList,
-      SqlNode offset,
-      SqlNode fetch) {
-    if (select.getOrderList() == null) {
-      assert collation.getFieldCollations().isEmpty();
-      if (offset == null && fetch == null) {
-        return;
-      }
-    }
-
-    // Create a sorter using the previously constructed collations.
-    bb.setRoot(
-        new SortRel(
-            cluster,
-            cluster.traitSetOf(Convention.NONE, collation),
-            bb.root,
-            collation,
-            offset == null ? null : convertExpression(offset),
-            fetch == null ? null : convertExpression(fetch)),
-        false);
-
-    // If extra expressions were added to the project list for sorting,
-    // add another project to remove them.
-    if (orderExprList.size() > 0) {
-      List<RexNode> exprs = new ArrayList<RexNode>();
-      final RelDataType rowType = bb.root.getRowType();
-      final int fieldCount =
-          rowType.getFieldCount() - orderExprList.size();
-      for (int i = 0; i < fieldCount; i++) {
-        exprs.add(rexBuilder.makeInputRef(bb.root, i));
-      }
-      bb.setRoot(
-          new ProjectRel(
-              cluster,
-              cluster.traitSetOf(RelCollationImpl.PRESERVE),
-              bb.root,
-              exprs,
-              cluster.getTypeFactory().createStructType(
-                  rowType.getFieldList().subList(0, fieldCount)),
-              ProjectRelBase.Flags.BOXED),
-          false);
-    }
-  }
-
-  /**
-   * Returns whether a given node contains a {@link SqlInOperator}.
-   *
-   * @param node a RexNode tree
-   */
-  private static boolean containsInOperator(
-      SqlNode node) {
-    try {
-      SqlVisitor<Void> visitor =
-          new SqlBasicVisitor<Void>() {
-            public Void visit(SqlCall call) {
-              if (call.getOperator() instanceof SqlInOperator) {
-                throw new Util.FoundOne(call);
-              }
-              return super.visit(call);
-            }
-          };
-      node.accept(visitor);
-      return false;
-    } catch (Util.FoundOne e) {
-      Util.swallow(e, null);
-      return true;
-    }
-  }
-
-  /**
-   * Push down all the NOT logical operators into any IN/NOT IN operators.
-   *
-   * @param sqlNode the root node from which to look for NOT operators
-   * @return the transformed SqlNode representation with NOT pushed down.
-   */
-  private static SqlNode pushDownNotForIn(SqlNode sqlNode) {
-    if ((sqlNode instanceof SqlCall) && containsInOperator(sqlNode)) {
-      SqlCall sqlCall = (SqlCall) sqlNode;
-      if ((sqlCall.getOperator() == SqlStdOperatorTable.AND)
-          || (sqlCall.getOperator() == SqlStdOperatorTable.OR)) {
-        SqlNode[] sqlOperands = ((SqlBasicCall) sqlCall).operands;
-        for (int i = 0; i < sqlOperands.length; i++) {
-          sqlOperands[i] = pushDownNotForIn(sqlOperands[i]);
-        }
-        return sqlNode;
-      } else if (sqlCall.getOperator() == SqlStdOperatorTable.NOT) {
-        SqlNode childNode = sqlCall.operand(0);
-        assert childNode instanceof SqlCall;
-        SqlBasicCall childSqlCall = (SqlBasicCall) childNode;
-        if (childSqlCall.getOperator() == SqlStdOperatorTable.AND) {
-          SqlNode[] andOperands = childSqlCall.getOperands();
-          SqlNode[] orOperands = new SqlNode[andOperands.length];
-          for (int i = 0; i < orOperands.length; i++) {
-            orOperands[i] =
-                SqlStdOperatorTable.NOT.createCall(
-                    SqlParserPos.ZERO,
-                    andOperands[i]);
-          }
-          for (int i = 0; i < orOperands.length; i++) {
-            orOperands[i] = pushDownNotForIn(orOperands[i]);
-          }
-          return SqlStdOperatorTable.OR.createCall(SqlParserPos.ZERO,
-              orOperands[0], orOperands[1]);
-        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.OR) {
-          SqlNode[] orOperands = childSqlCall.getOperands();
-          SqlNode[] andOperands = new SqlNode[orOperands.length];
-          for (int i = 0; i < andOperands.length; i++) {
-            andOperands[i] =
-                SqlStdOperatorTable.NOT.createCall(
-                    SqlParserPos.ZERO,
-                    orOperands[i]);
-          }
-          for (int i = 0; i < andOperands.length; i++) {
-            andOperands[i] = pushDownNotForIn(andOperands[i]);
-          }
-          return SqlStdOperatorTable.AND.createCall(SqlParserPos.ZERO,
-              andOperands[0], andOperands[1]);
-        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.NOT) {
-          SqlNode[] notOperands = childSqlCall.getOperands();
-          assert notOperands.length == 1;
-          return pushDownNotForIn(notOperands[0]);
-        } else if (childSqlCall.getOperator() instanceof SqlInOperator) {
-          SqlNode[] inOperands = childSqlCall.getOperands();
-          SqlInOperator inOp =
-              (SqlInOperator) childSqlCall.getOperator();
-          if (inOp.isNotIn()) {
-            return SqlStdOperatorTable.IN.createCall(
-                SqlParserPos.ZERO,
-                inOperands[0],
-                inOperands[1]);
-          } else {
-            return SqlStdOperatorTable.NOT_IN.createCall(
-                SqlParserPos.ZERO,
-                inOperands[0],
-                inOperands[1]);
-          }
-        } else {
-          // childSqlCall is "leaf" node in a logical expression tree
-          // (only considering AND, OR, NOT)
-          return sqlNode;
-        }
-      } else {
-        // sqlNode is "leaf" node in a logical expression tree
-        // (only considering AND, OR, NOT)
-        return sqlNode;
-      }
-    } else {
-      // tree rooted at sqlNode does not contain inOperator
-      return sqlNode;
-    }
-  }
-
-  /**
-   * Converts a WHERE clause.
-   *
-   * @param bb    Blackboard
-   * @param where WHERE clause, may be null
-   */
-  private void convertWhere(
-      final Blackboard bb,
-      final SqlNode where) {
-    if (where == null) {
-      return;
-    }
-    SqlNode newWhere = pushDownNotForIn(where);
-    replaceSubqueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
-    final RexNode convertedWhere = bb.convertExpression(newWhere);
-
-    // only allocate filter if the condition is not TRUE
-    if (!convertedWhere.isAlwaysTrue()) {
-      bb.setRoot(
-          RelOptUtil.createFilter(bb.root, convertedWhere),
-          false);
-    }
-  }
-
-  private void replaceSubqueries(
-      final Blackboard bb,
-      final SqlNode expr,
-      RelOptUtil.Logic logic) {
-    findSubqueries(bb, expr, logic, false);
-    for (SubQuery node : bb.subqueryList) {
-      substituteSubquery(bb, node);
-    }
-  }
-
-  private void substituteSubquery(Blackboard bb, SubQuery subQuery) {
-    final RexNode expr = subQuery.expr;
-    if (expr != null) {
-      // Already done.
-      return;
-    }
-
-    final SqlBasicCall call;
-    final RelNode rel;
-    final SqlNode query;
-    final Pair<RelNode, Boolean> converted;
-    switch (subQuery.node.getKind()) {
-    case CURSOR:
-      convertCursor(bb, subQuery);
-      return;
-
-    case MULTISET_QUERY_CONSTRUCTOR:
-    case MULTISET_VALUE_CONSTRUCTOR:
-      rel = convertMultisets(ImmutableList.of(subQuery.node), bb);
-      subQuery.expr = bb.register(rel, JoinRelType.INNER);
-      return;
-
-    case IN:
-      call = (SqlBasicCall) subQuery.node;
-      final SqlNode[] operands = call.getOperands();
-
-      SqlNode leftKeyNode = operands[0];
-      query = operands[1];
-
-      final List<RexNode> leftKeys;
-      switch (leftKeyNode.getKind()) {
-      case ROW:
-        leftKeys = Lists.newArrayList();
-        for (SqlNode sqlExpr : ((SqlBasicCall) leftKeyNode).getOperandList()) {
-          leftKeys.add(bb.convertExpression(sqlExpr));
-        }
-        break;
-      default:
-        leftKeys = ImmutableList.of(bb.convertExpression(leftKeyNode));
-      }
-
-      final boolean isNotIn = ((SqlInOperator) call.getOperator()).isNotIn();
-      if (query instanceof SqlNodeList) {
-        SqlNodeList valueList = (SqlNodeList) query;
-        if (!containsNullLiteral(valueList)
-            && valueList.size() < getInSubqueryThreshold()) {
-          // We're under the threshold, so convert to OR.
-          subQuery.expr =
-              convertInToOr(
-                  bb,
-                  leftKeys,
-                  valueList,
-                  isNotIn);
-          return;
-        }
-
-        // Otherwise, let convertExists translate
-        // values list into an inline table for the
-        // reference to Q below.
-      }
-
-      // Project out the search columns from the left side
-
-      //  Q1:
-      // "select from emp where emp.deptno in (select col1 from T)"
-      //
-      // is converted to
-      //
-      // "select from
-      //   emp inner join (select distinct col1 from T)) q
-      //   on emp.deptno = q.col1
-      //
-      // Q2:
-      // "select from emp where emp.deptno not in (Q)"
-      //
-      // is converted to
-      //
-      // "select from
-      //   emp left outer join (select distinct col1, TRUE from T) q
-      //   on emp.deptno = q.col1
-      //   where emp.deptno <> null
-      //         and q.indicator <> TRUE"
-      //
-      final boolean outerJoin = bb.subqueryNeedsOuterJoin
-          || isNotIn
-          || subQuery.logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
-      converted =
-          convertExists(query, RelOptUtil.SubqueryType.IN, subQuery.logic,
-              outerJoin);
-      if (converted.right) {
-        // Generate
-        //    emp CROSS JOIN (SELECT COUNT(*) AS c,
-        //                       COUNT(deptno) AS ck FROM dept)
-        final RelDataType longType =
-            typeFactory.createSqlType(SqlTypeName.BIGINT);
-        final RelNode seek = converted.left.getInput(0); // fragile
-        final int keyCount = leftKeys.size();
-        final List<Integer> args = ImmutableIntList.range(0, keyCount);
-        AggregateRel aggregate =
-            new AggregateRel(cluster, seek, BitSets.of(),
-                ImmutableList.of(
-                    new AggregateCall(SqlStdOperatorTable.COUNT, false,
-                        ImmutableList.<Integer>of(), longType, null),
-                    new AggregateCall(SqlStdOperatorTable.COUNT, false,
-                        args, longType, null)));
-        JoinRel join =
-            new JoinRel(cluster, bb.root, aggregate,
-                rexBuilder.makeLiteral(true), JoinRelType.INNER,
-                ImmutableSet.<String>of());
-        bb.setRoot(join, false);
-      }
-      RexNode rex =
-          bb.register(converted.left,
-              outerJoin ? JoinRelType.LEFT : JoinRelType.INNER, leftKeys);
-
-      subQuery.expr = translateIn(subQuery, bb.root, rex);
-      if (isNotIn) {
-        subQuery.expr =
-            rexBuilder.makeCall(SqlStdOperatorTable.NOT, subQuery.expr);
-      }
-      return;
-
-    case EXISTS:
-      // "select from emp where exists (select a from T)"
-      //
-      // is converted to the following if the subquery is correlated:
-      //
-      // "select from emp left outer join (select AGG_TRUE() as indicator
-      // from T group by corr_var) q where q.indicator is true"
-      //
-      // If there is no correlation, the expression is replaced with a
-      // boolean indicating whether the subquery returned 0 or >= 1 row.
-      call = (SqlBasicCall) subQuery.node;
-      query = call.getOperands()[0];
-      converted = convertExists(query, RelOptUtil.SubqueryType.EXISTS,
-          subQuery.logic, true);
-      assert !converted.right;
-      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, true)) {
-        return;
-      }
-      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
-      return;
-
-    case SCALAR_QUERY:
-      // Convert the subquery.  If it's non-correlated, convert it
-      // to a constant expression.
-      call = (SqlBasicCall) subQuery.node;
-      query = call.getOperands()[0];
-      converted = convertExists(query, RelOptUtil.SubqueryType.SCALAR,
-          subQuery.logic, true);
-      assert !converted.right;
-      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, false)) {
-        return;
-      }
-      rel = convertToSingleValueSubq(query, converted.left);
-      subQuery.expr = bb.register(rel, JoinRelType.LEFT);
-      return;
-
-    case SELECT:
-      // This is used when converting multiset queries:
-      //
-      // select * from unnest(select multiset[deptno] from emps);
-      //
-      converted = convertExists(subQuery.node, RelOptUtil.SubqueryType.SCALAR,
-          subQuery.logic, true);
-      assert !converted.right;
-      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
-      return;
-
-    default:
-      throw Util.newInternal("unexpected kind of subquery :" + subQuery.node);
-    }
-  }
-
-  private RexNode translateIn(SubQuery subQuery, RelNode root,
-      final RexNode rex) {
-    switch (subQuery.logic) {
-    case TRUE:
-      return rexBuilder.makeLiteral(true);
-
-    case UNKNOWN_AS_FALSE:
-      assert rex instanceof RexRangeRef;
-      final int fieldCount = rex.getType().getFieldCount();
-      RexNode rexNode = rexBuilder.makeFieldAccess(rex, fieldCount - 1);
-      rexNode = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, rexNode);
-
-      // Then append the IS NOT NULL(leftKeysForIn).
-      //
-      // RexRangeRef contains the following fields:
-      //   leftKeysForIn,
-      //   rightKeysForIn (the original subquery select list),
-      //   nullIndicator
-      //
-      // The first two lists contain the same number of fields.
-      final int k = (fieldCount - 1) / 2;
-      for (int i = 0; i < k; i++) {
-        rexNode =
-            rexBuilder.makeCall(
-                SqlStdOperatorTable.AND,
-                rexNode,
-                rexBuilder.makeCall(
-                    SqlStdOperatorTable.IS_NOT_NULL,
-                    rexBuilder.makeFieldAccess(rex, i)));
-      }
-      return rexNode;
-
-    case TRUE_FALSE_UNKNOWN:
-    case UNKNOWN_AS_TRUE:
-      // select e.deptno,
-      //   case
-      //   when ct.c = 0 then false
-      //   when dt.i is not null then true
-      //   when e.deptno is null then null
-      //   when ct.ck < ct.c then null
-      //   else false
-      //   end
-      // from e
-      // cross join (select count(*) as c, count(deptno) as ck from v) as ct
-      // left join (select distinct deptno, true as i from v) as dt
-      //   on e.deptno = dt.deptno
-      final JoinRelBase join = (JoinRelBase) root;
-      final ProjectRelBase left = (ProjectRelBase) join.getLeft();
-      final RelNode leftLeft = ((JoinRelBase) left.getInput(0)).getLeft();
-      final int leftLeftCount = leftLeft.getRowType().getFieldCount();
-      final RelDataType nullableBooleanType =
-          typeFactory.createTypeWithNullability(
-              typeFactory.createSqlType(SqlTypeName.BOOLEAN), true);
-      final RelDataType longType =
-          typeFactory.createSqlType(SqlTypeName.BIGINT);
-      final RexNode cRef = rexBuilder.makeInputRef(root, leftLeftCount);
-      final RexNode ckRef = rexBuilder.makeInputRef(root, leftLeftCount + 1);
-      final RexNode iRef =
-          rexBuilder.makeInputRef(root, root.getRowType().getFieldCount() - 1);
-
-      final RexLiteral zero =
-          rexBuilder.makeExactLiteral(BigDecimal.ZERO, longType);
-      final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
-      final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
-      final RexNode unknownLiteral =
-          rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
-
-      final ImmutableList.Builder<RexNode> args = ImmutableList.builder();
-      args.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, cRef, zero),
-          falseLiteral,
-          rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, iRef),
-          trueLiteral);
-      final JoinInfo joinInfo = join.analyzeCondition();
-      for (int leftKey : joinInfo.leftKeys) {
-        final RexNode kRef = rexBuilder.makeInputRef(root, leftKey);
-        args.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, kRef),
-            unknownLiteral);
-      }
-      args.add(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, ckRef, cRef),
-          unknownLiteral,
-          falseLiteral);
-
-      return rexBuilder.makeCall(
-          nullableBooleanType,
-          SqlStdOperatorTable.CASE,
-          args.build());
-
-    default:
-      throw new AssertionError(subQuery.logic);
-    }
-  }
-
-  private static boolean containsNullLiteral(SqlNodeList valueList) {
-    for (SqlNode node : valueList.getList()) {
-      if (node instanceof SqlLiteral) {
-        SqlLiteral lit = (SqlLiteral) node;
-        if (lit.getValue() == null) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Determines if a subquery is non-correlated and if so, converts it to a
-   * constant.
-   *
-   * @param subQuery  the call that references the subquery
-   * @param bb        blackboard used to convert the subquery
-   * @param converted RelNode tree corresponding to the subquery
-   * @param isExists  true if the subquery is part of an EXISTS expression
-   * @return if the subquery can be converted to a constant
-   */
-  private boolean convertNonCorrelatedSubQuery(
-      SubQuery subQuery,
-      Blackboard bb,
-      RelNode converted,
-      boolean isExists) {
-    SqlCall call = (SqlBasicCall) subQuery.node;
-    if (subqueryConverter.canConvertSubquery()
-        && isSubqNonCorrelated(converted, bb)) {
-      // First check if the subquery has already been converted
-      // because it's a nested subquery.  If so, don't re-evaluate
-      // it again.
-      RexNode constExpr = mapConvertedNonCorrSubqs.get(call);
-      if (constExpr == null) {
-        constExpr =
-            subqueryConverter.convertSubquery(
-                call,
-                this,
-                isExists,
-                isExplain);
-      }
-      if (constExpr != null) {
-        subQuery.expr = constExpr;
-        mapConvertedNonCorrSubqs.put(call, constExpr);
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Converts the RelNode tree for a select statement to a select that
-   * produces a single value.
-   *
-   * @param query the query
-   * @param plan   the original RelNode tree corresponding to the statement
-   * @return the converted RelNode tree
-   */
-  public RelNode convertToSingleValueSubq(
-      SqlNode query,
-      RelNode plan) {
-    // Check whether query is guaranteed to produce a single value.
-    if (query instanceof SqlSelect) {
-      SqlSelect select = (SqlSelect) query;
-      SqlNodeList selectList = select.getSelectList();
-      SqlNodeList groupList = select.getGroup();
-
-      if ((selectList.size() == 1)
-          && ((groupList == null) || (groupList.size() == 0))) {
-        SqlNode selectExpr = selectList.get(0);
-        if (selectExpr instanceof SqlCall) {
-          SqlCall selectExprCall = (SqlCall) selectExpr;
-          if (selectExprCall.getOperator()
-              instanceof SqlAggFunction) {
-            return plan;
-          }
-        }
-      }
-    }
-
-    // If not, project SingleValueAgg
-    return RelOptUtil.createSingleValueAggRel(
-        cluster,
-        plan);
-  }
-
-  /**
-   * Converts "x IN (1, 2, ...)" to "x=1 OR x=2 OR ...".
-   *
-   * @param leftKeys   LHS
-   * @param valuesList RHS
-   * @param isNotIn    is this a NOT IN operator
-   * @return converted expression
-   */
-  private RexNode convertInToOr(
-      final Blackboard bb,
-      final List<RexNode> leftKeys,
-      SqlNodeList valuesList,
-      boolean isNotIn) {
-    List<RexNode> comparisons = new ArrayList<RexNode>();
-    for (SqlNode rightVals : valuesList) {
-      RexNode rexComparison;
-      if (leftKeys.size() == 1) {
-        rexComparison =
-            rexBuilder.makeCall(
-                SqlStdOperatorTable.EQUALS,
-                leftKeys.get(0),
-                bb.convertExpression(rightVals));
-      } else {
-        assert rightVals instanceof SqlCall;
-        final SqlBasicCall call = (SqlBasicCall) rightVals;
-        assert (call.getOperator() instanceof SqlRowOperator)
-            && call.getOperands().length == leftKeys.size();
-        rexComparison =
-            RexUtil.composeConjunction(
-                rexBuilder,
-                Iterables.transform(
-                    Pair.zip(leftKeys, call.getOperandList()),
-                    new Function<Pair<RexNode, SqlNode>, RexNode>() {
-                      public RexNode apply(Pair<RexNode, SqlNode> pair) {
-                        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                            pair.left, bb.convertExpression(pair.right));
-                      }
-                    }),
-                false);
-      }
-      comparisons.add(rexComparison);
-    }
-
-    RexNode result =
-        RexUtil.composeDisjunction(rexBuilder, comparisons, true);
-    assert result != null;
-
-    if (isNotIn) {
-      result =
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.NOT,
-              result);
-    }
-
-    return result;
-  }
-
-  /**
-   * Gets the list size threshold under which {@link #convertInToOr} is used.
-   * Lists of this size or greater will instead be converted to use a join
-   * against an inline table ({@link ValuesRel}) rather than a predicate. A
-   * threshold of 0 forces usage of an inline table in all cases; a threshold
-   * of Integer.MAX_VALUE forces usage of OR in all cases
-   *
-   * @return threshold, default 20
-   */
-  protected int getInSubqueryThreshold() {
-    // OVERRIDE POINT
-    return Integer.MAX_VALUE; // was 20
-  }
-
-  /**
-   * Converts an EXISTS or IN predicate into a join. For EXISTS, the subquery
-   * produces an indicator variable, and the result is a relational expression
-   * which outer joins that indicator to the original query. After performing
-   * the outer join, the condition will be TRUE if the EXISTS condition holds,
-   * NULL otherwise.
-   *
-   * @param seek           A query, for example 'select * from emp' or
-   *                       'values (1,2,3)' or '('Foo', 34)'.
-   * @param subqueryType   Whether sub-query is IN, EXISTS or scalar
-   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
-   *     FALSE, UNKNOWN) will be required, or whether we can accept an
-   *     approximation (say representing UNKNOWN as FALSE)
-   * @param needsOuterJoin Whether an outer join is needed
-   * @return join expression
-   * @pre extraExpr == null || extraName != null
-   */
-  private Pair<RelNode, Boolean> convertExists(
-      SqlNode seek,
-      RelOptUtil.SubqueryType subqueryType,
-      RelOptUtil.Logic logic,
-      boolean needsOuterJoin) {
-    final SqlValidatorScope seekScope =
-        (seek instanceof SqlSelect)
-            ? validator.getSelectScope((SqlSelect) seek)
-            : null;
-    final Blackboard seekBb = createBlackboard(seekScope, null);
-    RelNode seekRel = convertQueryOrInList(seekBb, seek);
-
-    return RelOptUtil.createExistsPlan(seekRel, subqueryType, logic,
-        needsOuterJoin);
-  }
-
-  private RelNode convertQueryOrInList(
-      Blackboard bb,
-      SqlNode seek) {
-    // NOTE: Once we start accepting single-row queries as row constructors,
-    // there will be an ambiguity here for a case like X IN ((SELECT Y FROM
-    // Z)).  The SQL standard resolves the ambiguity by saying that a lone
-    // select should be interpreted as a table expression, not a row
-    // expression.  The semantic difference is that a table expression can
-    // return multiple rows.
-    if (seek instanceof SqlNodeList) {
-      return convertRowValues(
-          bb,
-          seek,
-          ((SqlNodeList) seek).getList(),
-          false,
-          null);
-    } else {
-      return convertQueryRecursive(seek, false, null);
-    }
-  }
-
-  private RelNode convertRowValues(
-      Blackboard bb,
-      SqlNode rowList,
-      Collection<SqlNode> rows,
-      boolean allowLiteralsOnly,
-      RelDataType targetRowType) {
-    // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of
-    // literals into a single ValuesRel; this gives the optimizer a smaller
-    // input tree.  For everything else (computed expressions, row
-    // subqueries), we union each row in as a projection on top of a
-    // OneRowRel.
-
-    final List<List<RexLiteral>> tupleList =
-        new ArrayList<List<RexLiteral>>();
-    final RelDataType rowType;
-    if (targetRowType != null) {
-      rowType = targetRowType;
-    } else {
-      rowType =
-          SqlTypeUtil.promoteToRowType(
-              typeFactory,
-              validator.getValidatedNodeType(rowList),
-              null);
-    }
-
-    List<RelNode> unionInputs = new ArrayList<RelNode>();
-    for (SqlNode node : rows) {
-      SqlBasicCall call;
-      if (isRowConstructor(node)) {
-        call = (SqlBasicCall) node;
-        List<RexLiteral> tuple = new ArrayList<RexLiteral>();
-        for (SqlNode operand : call.operands) {
-          RexLiteral rexLiteral =
-              convertLiteralInValuesList(
-                  operand,
-                  bb,
-                  rowType,
-                  tuple.size());
-          if ((rexLiteral == null) && allowLiteralsOnly) {
-            return null;
-          }
-          if ((rexLiteral == null) || !shouldCreateValuesRel) {
-            // fallback to convertRowConstructor
-            tuple = null;
-            break;
-          }
-          tuple.add(rexLiteral);
-        }
-        if (tuple != null) {
-          tupleList.add(tuple);
-          continue;
-        }
-      } else {
-        RexLiteral rexLiteral =
-            convertLiteralInValuesList(
-                node,
-                bb,
-                rowType,
-                0);
-        if ((rexLiteral != null) && shouldCreateValuesRel) {
-          tupleList.add(
-              Collections.singletonList(rexLiteral));
-          continue;
-        } else {
-          if ((rexLiteral == null) && allowLiteralsOnly) {
-            return null;
-          }
-        }
-
-        // convert "1" to "row(1)"
-        call =
-            (SqlBasicCall) SqlStdOperatorTable.ROW.createCall(
-                SqlParserPos.ZERO,
-                node);
-      }
-      unionInputs.add(convertRowConstructor(bb, call));
-    }
-    ValuesRel valuesRel =
-        new ValuesRel(
-            cluster,
-            rowType,
-            tupleList);
-    RelNode resultRel;
-    if (unionInputs.isEmpty()) {
-      resultRel = valuesRel;
-    } else {
-      if (!tupleList.isEmpty()) {
-        unionInputs.add(valuesRel);
-      }
-      UnionRel unionRel =
-          new UnionRel(
-              cluster,
-              unionInputs,
-              true);
-      resultRel = unionRel;
-    }
-    leaves.add(resultRel);
-    return resultRel;
-  }
-
-  private RexLiteral convertLiteralInValuesList(
-      SqlNode sqlNode,
-      Blackboard bb,
-      RelDataType rowType,
-      int iField) {
-    if (!(sqlNode instanceof SqlLiteral)) {
-      return null;
-    }
-    RelDataTypeField field = rowType.getFieldList().get(iField);
-    RelDataType type = field.getType();
-    if (type.isStruct()) {
-      // null literals for weird stuff like UDT's need
-      // special handling during type flattening, so
-      // don't use ValuesRel for those
-      return null;
-    }
-
-    RexNode literalExpr =
-        exprConverter.convertLiteral(
-            bb,
-            (SqlLiteral) sqlNode);
-
-    if (!(literalExpr instanceof RexLiteral)) {
-      assert literalExpr.isA(SqlKind.CAST);
-      RexNode child = ((RexCall) literalExpr).getOperands().get(0);
-      assert RexLiteral.isNullLiteral(child);
-
-      // NOTE jvs 22-Nov-2006:  we preserve type info
-      // in ValuesRel digest, so it's OK to lose it here
-      return (RexLiteral) child;
-    }
-
-    RexLiteral literal = (RexLiteral) literalExpr;
-
-    Comparable value = literal.getValue();
-
-    if (SqlTypeUtil.isExactNumeric(type)) {
-      BigDecimal roundedValue =
-          NumberUtil.rescaleBigDecimal(
-              (BigDecimal) value,
-              type.getScale());
-      return rexBuilder.makeExactLiteral(
-          roundedValue,
-          type);
-    }
-
-    if ((value instanceof NlsString)
-        && (type.getSqlTypeName() == SqlTypeName.CHAR)) {
-      // pad fixed character type
-      NlsString unpadded = (NlsString) value;
-      return rexBuilder.makeCharLiteral(
-          new NlsString(
-              Util.rpad(unpadded.getValue(), type.getPrecision()),
-              unpadded.getCharsetName(),
-              unpadded.getCollation()));
-    }
-    return literal;
-  }
-
-  private boolean isRowConstructor(SqlNode node) {
-    if (!(node.getKind() == SqlKind.ROW)) {
-      return false;
-    }
-    SqlCall call = (SqlCall) node;
-    return call.getOperator().getName().equalsIgnoreCase("row");
-  }
-
-  /**
-   * Builds a list of all <code>IN</code> or <code>EXISTS</code> operators
-   * inside SQL parse tree. Does not traverse inside queries.
-   *
-   * @param bb                           blackboard
-   * @param node                         the SQL parse tree
-   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
-   *              FALSE, UNKNOWN) will be required, or whether we can accept
-   *              an approximation (say representing UNKNOWN as FALSE)
-   * @param registerOnlyScalarSubqueries if set to true and the parse tree
-   *                                     corresponds to a variation of a select
-   *                                     node, only register it if it's a scalar
-   *                                     subquery
-   */
-  private void findSubqueries(
-      Blackboard bb,
-      SqlNode node,
-      RelOptUtil.Logic logic,
-      boolean registerOnlyScalarSubqueries) {
-    final SqlKind kind = node.getKind();
-    switch (kind) {
-    case EXISTS:
-    case SELECT:
-    case MULTISET_QUERY_CONSTRUCTOR:
-    case MULTISET_VALUE_CONSTRUCTOR:
-    case CURSOR:
-    case SCALAR_QUERY:
-      if (!registerOnlyScalarSubqueries
-          || (kind == SqlKind.SCALAR_QUERY)) {
-        bb.registerSubquery(node, RelOptUtil.Logic.TRUE_FALSE);
-      }
-      return;
-    case IN:
-      if (((SqlCall) node).getOperator() == SqlStdOperatorTable.NOT_IN) {
-        logic = logic.negate();
-      }
-      break;
-    case NOT:
-      logic = logic.negate();
-      break;
-    }
-    if (node instanceof SqlCall) {
-      if (kind == SqlKind.OR
-          || kind == SqlKind.NOT) {
-        // It's always correct to outer join subquery with
-        // containing query; however, when predicates involve Or
-        // or NOT, outer join might be necessary.
-        bb.subqueryNeedsOuterJoin = true;
-      }
-      for (SqlNode operand : ((SqlCall) node).getOperandList()) {
-        if (operand != null) {
-          // In the case of an IN expression, locate scalar
-          // subqueries so we can convert them to constants
-          findSubqueries(
-              bb,
-              operand,
-              logic,
-              kind == SqlKind.IN || registerOnlyScalarSubqueries);
-        }
-      }
-    } else if (node instanceof SqlNodeList) {
-      for (SqlNode child : (SqlNodeList) node) {
-        findSubqueries(
-            bb,
-            child,
-            logic,
-            kind == SqlKind.IN || registerOnlyScalarSubqueries);
-      }
-    }
-
-    // Now that we've located any scalar subqueries inside the IN
-    // expression, register the IN expression itself.  We need to
-    // register the scalar subqueries first so they can be converted
-    // before the IN expression is converted.
-    if (kind == SqlKind.IN) {
-      if (logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN
-          && !validator.getValidatedNodeType(node).isNullable()) {
-        logic = RelOptUtil.Logic.UNKNOWN_AS_FALSE;
-      }
-      // TODO: This conversion is only valid in the WHERE clause
-      if (logic == RelOptUtil.Logic.UNKNOWN_AS_FALSE
-          && !bb.subqueryNeedsOuterJoin) {
-        logic = RelOptUtil.Logic.TRUE;
-      }
-      bb.registerSubquery(node, logic);
-    }
-  }
-
-  /**
-   * Converts an expression from {@link SqlNode} to {@link RexNode} format.
-   *
-   * @param node Expression to translate
-   * @return Converted expression
-   */
-  public RexNode convertExpression(
-      SqlNode node) {
-    Map<String, RelDataType> nameToTypeMap = Collections.emptyMap();
-    Blackboard bb =
-        createBlackboard(
-            new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap),
-            null);
-    return bb.convertExpression(node);
-  }
-
-  /**
-   * Converts an expression from {@link SqlNode} to {@link RexNode} format,
-   * mapping identifier references to predefined expressions.
-   *
-   * @param node          Expression to translate
-   * @param nameToNodeMap map from String to {@link RexNode}; when an
-   *                      {@link SqlIdentifier} is encountered, it is used as a
-   *                      key and translated to the corresponding value from
-   *                      this map
-   * @return Converted expression
-   */
-  public RexNode convertExpression(
-      SqlNode node,
-      Map<String, RexNode> nameToNodeMap) {
-    final Map<String, RelDataType> nameToTypeMap =
-        new HashMap<String, RelDataType>();
-    for (Map.Entry<String, RexNode> entry : nameToNodeMap.entrySet()) {
-      nameToTypeMap.put(entry.getKey(), entry.getValue().getType());
-    }
-    Blackboard bb =
-        createBlackboard(
-            new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap),
-            nameToNodeMap);
-    return bb.convertExpression(node);
-  }
-
-  /**
-   * Converts a non-standard expression.
-   *
-   * <p>This method is an extension-point that derived classes can override. If
-   * this method returns a null result, the normal expression translation
-   * process will proceed. The default implementation always returns null.
-   *
-   * @param node Expression
-   * @param bb   Blackboard
-   * @return null to proceed with the usual expression translation process
-   */
-  protected RexNode convertExtendedExpression(
-      SqlNode node,
-      Blackboard bb) {
-    return null;
-  }
-
-  private RexNode convertOver(Blackboard bb, SqlNode node) {
-    SqlCall call = (SqlCall) node;
-    SqlCall aggCall = call.operand(0);
-    SqlNode windowOrRef = call.operand(1);
-    final SqlWindow window =
-        validator.resolveWindow(windowOrRef, bb.scope, true);
-    final SqlNodeList partitionList = window.getPartitionList();
-    final ImmutableList.Builder<RexNode> partitionKeys =
-        ImmutableList.builder();
-    for (SqlNode partition : partitionList) {
-      partitionKeys.add(bb.convertExpression(partition));
-    }
-    RexNode lowerBound = bb.convertExpression(window.getLowerBound());
-    RexNode upperBound = bb.convertExpression(window.getUpperBound());
-    SqlNodeList orderList = window.getOrderList();
-    if ((orderList.size() == 0) && !window.isRows()) {
-      // A logical range requires an ORDER BY clause. Use the implicit
-      // ordering of this relation. There must be one, otherwise it would
-      // have failed validation.
-      orderList = bb.scope.getOrderList();
-      if (orderList == null) {
-        throw new AssertionError(
-            "Relation should have sort key for implicit ORDER BY");
-      }
-    }
-    final ImmutableList.Builder<RexFieldCollation> orderKeys =
-        ImmutableList.builder();
-    final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
-    for (SqlNode order : orderList) {
-      flags.clear();
-      RexNode e = bb.convertSortExpression(order, flags);
-      orderKeys.add(new RexFieldCollation(e, flags));
-    }
-    try {
-      Util.permAssert(bb.window == null, "already in window agg mode");
-      bb.window = window;
-      RexNode rexAgg = exprConverter.convertCall(bb, aggCall);
-      rexAgg =
-          rexBuilder.ensureType(
-              validator.getValidatedNodeType(call), rexAgg, false);
-
-      // Walk over the tree and apply 'over' to all agg functions. This is
-      // necessary because the returned expression is not necessarily a call
-      // to an agg function. For example, AVG(x) becomes SUM(x) / COUNT(x).
-      final RexShuttle visitor =
-          new HistogramShuttle(
-              partitionKeys.build(), orderKeys.build(),
-              RexWindowBound.create(window.getLowerBound(), lowerBound),
-              RexWindowBound.create(window.getUpperBound(), upperBound),
-              window);
-      return rexAgg.accept(visitor);
-    } finally {
-      bb.window = null;
-    }
-  }
-
-  /**
-   * Converts a FROM clause into a relational expression.
-   *
-   * @param bb   Scope within which to resolve identifiers
-   * @param from FROM clause of a query. Examples include:
-   *
-   *             <ul>
-   *             <li>a single table ("SALES.EMP"),
-   *             <li>an aliased table ("EMP AS E"),
-   *             <li>a list of tables ("EMP, DEPT"),
-   *             <li>an ANSI Join expression ("EMP JOIN DEPT ON EMP.DEPTNO =
-   *             DEPT.DEPTNO"),
-   *             <li>a VALUES clause ("VALUES ('Fred', 20)"),
-   *             <li>a query ("(SELECT * FROM EMP WHERE GENDER = 'F')"),
-   *             <li>or any combination of the above.
-   *             </ul>
-   */
-  protected void convertFrom(
-      Blackboard bb,
-      SqlNode from) {
-    SqlCall call;
-    final SqlNode[] operands;
-    switch (from.getKind()) {
-    case AS:
-      operands = ((SqlBasicCall) from).getOperands();
-      convertFrom(bb, operands[0]);
-      return;
-
-    case WITH_ITEM:
-      convertFrom(bb, ((SqlWithItem) from).query);
-      return;
-
-    case WITH:
-      convertFrom(bb, ((SqlWith) from).body);
-      return;
-
-    case TABLESAMPLE:
-      operands = ((SqlBasicCall) from).getOperands();
-      SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands[1]);
-      if (sampleSpec instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) {
-        String sampleName =
-            ((SqlSampleSpec.SqlSubstitutionSampleSpec) sampleSpec)
-                .getName();
-        datasetStack.push(sampleName);
-        convertFrom(bb, operands[0]);
-        datasetStack.pop();
-      } else if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) {
-        SqlSampleSpec.SqlTableSampleSpec tableSampleSpec =
-            (SqlSampleSpec.SqlTableSampleSpec) sampleSpec;
-        convertFrom(bb, operands[0]);
-        RelOptSamplingParameters params =
-            new RelOptSamplingParameters(
-                tableSampleSpec.isBernoulli(),
-                tableSampleSpec.getSamplePercentage(),
-                tableSampleSpec.isRepeatable(),
-                tableSampleSpec.getRepeatableSeed());
-        bb.setRoot(new SamplingRel(cluster, bb.root, params), false);
-      } else {
-        throw Util.newInternal(
-            "unknown TABLESAMPLE type: " + sampleSpec);
-      }
-      return;
-
-    case IDENTIFIER:
-      final SqlValidatorNamespace fromNamespace =
-          validator.getNamespace(from).resolve();
-      if (fromNamespace.getNode() != null) {
-        convertFrom(bb, fromNamespace.getNode());
-        return;
-      }
-      final String datasetName =
-          datasetStack.isEmpty() ? null : datasetStack.peek();
-      boolean[] usedDataset = {false};
-      RelOptTable table =
-          SqlValidatorUtil.getRelOptTable(
-              fromNamespace,
-              catalogReader,
-              datasetName,
-              usedDataset);
-      final RelNode tableRel;
-      if (shouldConvertTableAccess) {
-        tableRel = toRel(table);
-      } else {
-        tableRel = new TableAccessRel(cluster, table);
-      }
-      bb.setRoot(tableRel, true);
-      if (usedDataset[0]) {
-        bb.setDataset(datasetName);
-      }
-      return;
-
-    case JOIN:
-      final SqlJoin join = (SqlJoin) from;
-      final Blackboard fromBlackboard =
-          createBlackboard(validator.getJoinScope(from), null);
-      SqlNode left = join.getLeft();
-      SqlNode right = join.getRight();
-      final boolean isNatural = join.isNatural();
-      final JoinType joinType = join.getJoinType();
-      final Blackboard leftBlackboard =
-          createBlackboard(
-              Util.first(validator.getJoinScope(left),
-                  ((DelegatingScope) bb.scope).getParent()), null);
-      final Blackboard rightBlackboard =
-          createBlackboard(
-              Util.first(validator.getJoinScope(right),
-                  ((DelegatingScope) bb.scope).getParent()), null);
-      convertFrom(leftBlackboard, left);
-      RelNode leftRel = leftBlackboard.root;
-      convertFrom(rightBlackboard, right);
-      RelNode rightRel = rightBlackboard.root;
-      JoinRelType convertedJoinType = convertJoinType(joinType);
-      RexNode conditionExp;
-      if (isNatural) {
-        final List<String> columnList =
-            SqlValidatorUtil.deriveNaturalJoinColumnList(
-                validator.getNamespace(left).getRowType(),
-                validator.getNamespace(right).getRowType());
-        conditionExp = convertUsing(leftRel, rightRel, columnList);
-      } else {
-        conditionExp =
-            convertJoinCondition(
-                fromBlackboard,
-                join.getCondition(),
-                join.getConditionType(),
-                leftRel,
-                rightRel);
-      }
-
-      final RelNode joinRel =
-          createJoin(
-              fromBlackboard,
-              leftRel,
-              rightRel,
-              conditionExp,
-              convertedJoinType);
-      bb.setRoot(joinRel, false);
-      return;
-
-    case SELECT:
-    case INTERSECT:
-    case EXCEPT:
-    case UNION:
-      final RelNode rel = convertQueryRecursive(from, false, null);
-      bb.setRoot(rel, true);
-      return;
-
-    case VALUES:
-      convertValuesImpl(bb, (SqlCall) from, null);
-      return;
-
-    case UNNEST:
-      final SqlNode node = ((SqlCall) from).operand(0);
-      replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
-      final RelNode childRel =
-          RelOptUtil.createProject(
-              (null != bb.root) ? bb.root : new OneRowRel(cluster),
-              Collections.singletonList(bb.convertExpression(node)),
-              Collections.singletonList(validator.deriveAlias(node, 0)),
-              true);
-
-      UncollectRel uncollectRel =
-          new UncollectRel(cluster, cluster.traitSetOf(Convention.NONE),
-              childRel);
-      bb.setRoot(uncollectRel, true);
-      return;
-
-    case COLLECTION_TABLE:
-      call = (SqlCall) from;
-
-      // Dig out real call; TABLE() wrapper is just syntactic.
-      assert call.getOperandList().size() == 1;
-      call = call.operand(0);
-      convertCollectionTable(bb, call);
-      return;
-
-    default:
-      throw Util.newInternal("not a join operator " + from);
-    }
-  }
-
-  protected void convertCollectionTable(
-      Blackboard bb,
-      SqlCall call) {
-    final SqlOperator operator = call.getOperator();
-    if (operator == SqlStdOperatorTable.TABLESAMPLE) {
-      final String sampleName =
-          SqlLiteral.stringValue(call.operand(0));
-      datasetStack.push(sampleName);
-      SqlCall cursorCall = call.operand(1);
-      SqlNode query = cursorCall.operand(0);
-      RelNode converted = convertQuery(query, false, false);
-      bb.setRoot(converted, false);
-      datasetStack.pop();
-      return;
-    }
-    replaceSubqueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
-
-    // Expand table macro if possible. It's more efficient than
-    // TableFunctionRel.
-    if (operator instanceof SqlUserDefinedTableMacro) {
-      final SqlUserDefinedTableMacro udf =
-          (SqlUserDefinedTableMacro) operator;
-      final TranslatableTable table = udf.getTable(typeFactory,
-        call.getOperandList());
-      final RelDataType rowType = table.getRowType(typeFactory);
-      RelOptTable relOptTable = RelOptTableImpl.create(null, rowType, table);
-      RelNode converted = toRel(relOptTable);
-      bb.setRoot(converted, true);
-      return;
-    }
-
-    Type elementType;
-    if (operator instanceof SqlUserDefinedTableFunction) {
-      SqlUserDefinedTableFunction udtf = (SqlUserDefinedTableFunction) operator;
-      elementType = udtf.getElementType(typeFactory, call.getOperandList());
-    } else {
-      elementType = null;
-    }
-
-    RexNode rexCall = bb.convertExpression(call);
-    final List<RelNode> inputs = bb.retrieveCursors();
-    Set<RelColumnMapping> columnMappings =
-        getColumnMappings(operator);
-    TableFunctionRel callRel =
-        new TableFunctionRel(
-            cluster,
-            inputs,
-            rexCall,
-            elementType,
-            validator.getValidatedNodeType(call),
-            columnMappings);
-    bb.setRoot(callRel, true);
-    afterTableFunction(bb, call, callRel);
-  }
-
-  protected void afterTableFunction(
-      SqlToRelConverter.Blackboard bb,
-      SqlCall call,
-      TableFunctionRel callRel) {
-  }
-
-  private Set<RelColumnMapping> getColumnMappings(SqlOperator op) {
-    SqlReturnTypeInference rti = op.getReturnTypeInference();
-    if (rti == null) {
-      return null;
-    }
-    if (rti instanceof TableFunctionReturnTypeInference) {
-      TableFunctionReturnTypeInference tfrti =
-          (TableFunctionReturnTypeInference) rti;
-      return tfrti.getColumnMappings();
-    } else {
-      return null;
-    }
-  }
-
-  protected RelNode createJoin(
-      Blackboard bb,
-      RelNode leftRel,
-      RelNode rightRel,
-      RexNode joinCond,
-      JoinRelType joinType) {
-    assert joinCond != null;
-
-    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(rightRel);
-    if (correlatedVariables.size() > 0) {
-      final List<Correlation> correlations = Lists.newArrayList();
-
-      for (String correlName : correlatedVariables) {
-        DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
-        RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName);
-        String originalRelName = lookup.getOriginalRelName();
-        String originalFieldName = fieldAccess.getField().getName();
-
-        int[] nsIndexes = {-1};
-        final SqlValidatorScope[] ancestorScopes = {null};
-        SqlValidatorNamespace foundNs =
-            lookup.bb.scope.resolve(
-                originalRelName,
-                ancestorScopes,
-                nsIndexes);
-
-        assert foundNs != null;
-        assert nsIndexes.length == 1;
-
-        int childNamespaceIndex = nsIndexes[0];
-
-        SqlValidatorScope ancestorScope = ancestorScopes[0];
-        boolean correlInCurrentScope = ancestorScope == bb.scope;
-
-        if (correlInCurrentScope) {
-          int namespaceOffset = 0;
-          if (childNamespaceIndex > 0) {
-            // If not the first child, need to figure out the width
-            // of output types from all the preceding namespaces
-            assert ancestorScope instanceof ListScope;
-            List<SqlValidatorNamespace> children =
-                ((ListScope) ancestorScope).getChildren();
-
-            for (int i = 0; i < childNamespaceIndex; i++) {
-              SqlValidatorNamespace child = children.get(i);
-              namespaceOffset +=
-                  child.getRowType().getFieldCount();
-            }
-          }
-
-          RelDataTypeField field =
-              catalogReader.field(foundNs.getRowType(), originalFieldName);
-          int pos = namespaceOffset + field.getIndex();
-
-          assert field.getType()
-              == lookup.getFieldAccess(correlName).getField().getType();
-
-          assert pos != -1;
-
-          if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) {
-            // bb.root is an aggregate and only projects group by
-            // keys.
-            Map<Integer, Integer> exprProjection =
-                bb.mapRootRelToFieldProjection.get(bb.root);
-
-            // subquery can reference group by keys projected from
-            // the root of the outer relation.
-            if (exprProjection.containsKey(pos)) {
-              pos = exprProjection.get(pos);
-            } else {
-              // correl not grouped
-              throw Util.newInternal(
-                  "Identifier '" + originalRelName + "."
-                  + originalFieldName + "' is not a group expr");
-            }
-          }
-
-          Correlation newCorVar =
-              new Correlation(
-                  getCorrelOrdinal(correlName),
-                  pos);
-
-          correlations.add(newCorVar);
-        }
-      }
-
-      if (!correlations.isEmpty()) {
-        return new CorrelatorRel(
-            rightRel.getCluster(),
-            leftRel,
-            rightRel,
-            joinCond,
-            correlations,
-            joinType);
-      }
-    }
-
-    final List<RexNode> extraLeftExprs = new ArrayList<RexNode>();
-    final List<RexNode> extraRightExprs = new ArrayList<RexNode>();
-    final int leftCount = leftRel.getRowType().getFieldCount();
-    final int rightCount = rightRel.getRowType().getFieldCount();
-    if (!containsGet(joinCond)) {
-      joinCond = pushDownJoinConditions(
-          joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs);
-    }
-    if (!extraLeftExprs.isEmpty()) {
-      final List<RelDataTypeField> fields =
-          leftRel.getRowType().getFieldList();
-      leftRel = RelOptUtil.createProject(
-          leftRel,
-          new AbstractList<Pair<RexNode, String>>() {
-            @Override
-            public int size() {
-              return leftCount + extraLeftExprs.size();
-            }
-
-            @Override
-            public Pair<RexNode, String> get(int index) {
-              if (index < leftCount) {
-                RelDataTypeField field = fields.get(index);
-                return Pair.<RexNode, String>of(
-                    new RexInputRef(index, field.getType()),
-                    field.getName());
-              } else {
-                return Pair.<RexNode, String>of(
-                    extraLeftExprs.get(index - leftCount), null);
-              }
-            }
-          },
-          true);
-    }
-    if (!extraRightExprs.isEmpty()) {
-      final List<RelDataTypeField> fields =
-          rightRel.getRowType().getFieldList();
-      final int newLeftCount = leftCount + extraLeftExprs.size();
-      rightRel = RelOptUtil.createProject(
-          rightRel,
-          new AbstractList<Pair<RexNode, String>>() {
-            @Override
-            public int size() {
-              return rightCount + extraRightExprs.size();
-            }
-
-            @Override
-            public Pair<RexNode, String> get(int index) {
-              if (index < rightCount) {
-                RelDataTypeField field = fields.get(index);
-                return Pair.<RexNode, String>of(
-                    new RexInputRef(index, field.getType()),
-                    field.getName());
-              } else {
-                return Pair.of(
-                    RexUtil.shift(
-                        extraRightExprs.get(index - rightCount),
-                        -newLeftCount),
-                    null);
-              }
-            }
-          },
-          true);
-    }
-    RelNode join = createJoin(
-        leftRel,
-        rightRel,
-        joinCond,
-        joinType,
-        ImmutableSet.<String>of());
-    if (!extraLeftExprs.isEmpty() || !extraRightExprs.isEmpty()) {
-      Mappings.TargetMapping mapping =
-          Mappings.createShiftMapping(
-              leftCount + extraLeftExprs.size()
-                  + rightCount + extraRightExprs.size(),
-              0, 0, leftCount,
-              leftCount, leftCount + extraLeftExprs.size(), rightCount);
-      return RelOptUtil.project(join, mapping);
-    }
-    return join;
-  }
-
-  private static boolean containsGet(RexNode node) {
-    try {
-      node.accept(
-          new RexVisitorImpl<Void>(true) {
-            @Override public Void visitCall(RexCall call) {
-              if (call.getOperator() == RexBuilder.GET_OPERATOR) {
-                throw Util.FoundOne.NULL;
-              }
-              return super.visitCall(call);
-            }
-          });
-      return false;
-    } catch (Util.FoundOne e) {
-      return true;
-    }
-  }
-
-  /**
-   * Pushes down parts of a join condition. For example, given
-   * "emp JOIN dept ON emp.deptno + 1 = dept.deptno", adds a project above
-   * "emp" that computes the expression
-   * "emp.deptno + 1". The resulting join condition is a simple combination
-   * of AND, equals, and input fields.
-   */
-  private RexNode pushDownJoinConditions(
-      RexNode node,
-      int leftCount,
-      int rightCount,
-      List<RexNode> extraLeftExprs,
-      List<RexNode> extraRightExprs) {
-    switch (node.getKind()) {
-    case AND:
-    case OR:
-    case EQUALS:
-      RexCall call = (RexCall) node;
-      List<RexNode> list = new ArrayList<RexNode>();
-      List<RexNode> operands = Lists.newArrayList(call.getOperands());
-      for (int i = 0; i < operands.size(); i++) {
-        RexNode operand = operands.get(i);
-        final int left2 = leftCount + extraLeftExprs.size();
-        final int right2 = rightCount + extraRightExprs.size();
-        final RexNode e =
-            pushDownJoinConditions(
-                operand,
-                leftCount,
-                rightCount,
-                extraLeftExprs,
-                extraRightExprs);
-        final List<RexNode> remainingOperands = Util.skip(operands, i + 1);
-        final int left3 = leftCount + extraLeftExprs.size();
-        final int right3 = rightCount + extraRightExprs.size();
-        fix(remainingOperands, left2, left3);
-        fix(list, left2, left3);
-        list.add(e);
-      }
-      if (!list.equals(call.getOperands())) {
-        return call.clone(call.getType(), list);
-      }
-      return call;
-    case INPUT_REF:
-    case LITERAL:
-      return node;
-    default:
-      BitSet bits = RelOptUtil.InputFinder.bits(node);
-      final int mid = leftCount + extraLeftExprs.size();
-      switch (Side.of(bits, mid)) {
-      case LEFT:
-        fix(extraRightExprs, mid, mid + 1);
-        extraLeftExprs.add(node);
-        return new RexInputRef(mid, node.getType());
-      case RIGHT:
-        final int index2 = mid + rightCount + extraRightExprs.size();
-        extraRightExprs.add(node);
-        return new RexInputRef(index2, node.getType());
-      case BOTH:
-      case EMPTY:
-      default:
-        return node;
-      }
-    }
-  }
-
-  private void fix(List<RexNode> operands, int before, int after) {
-    if (before == after) {
-      return;
-    }
-    for (int i = 0; i < operands.size(); i++) {
-      RexNode node = operands.get(i);
-      operands.set(i, RexUtil.shift(node, before, after - before));
-    }
-  }
-
-  /**
-   * Categorizes whether a bit set contains bits left and right of a
-   * line.
-   */
-  enum Side {
-    LEFT, RIGHT, BOTH, EMPTY;
-
-    static Side of(BitSet bitSet, int middle) {
-      final int firstBit = bitSet.nextSetBit(0);
-      if (firstBit < 0) {
-        return EMPTY;
-      }
-      if (firstBit >= middle) {
-        return RIGHT;
-      }
-      if (bitSet.nextSetBit(middle) < 0) {
-        return LEFT;
-      }
-      return BOTH;
-    }
-  }
-
-  /**
-   * Determines whether a subquery is non-correlated. Note that a
-   * non-correlated subquery can contain correlated references, provided those
-   * references do not reference select statements that are parents of the
-   * subquery.
-   *
-   * @param subq the subquery
-   * @param bb   blackboard used while converting the subquery, i.e., the
-   *             blackboard of the parent query of this subquery
-   * @return true if the subquery is non-correlated.
-   */
-  private boolean isSubqNonCorrelated(RelNode subq, Blackboard bb) {
-    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(subq);
-    for (String correlName : correlatedVariables) {
-      DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
-      String originalRelName = lookup.getOriginalRelName();
-
-      int[] nsIndexes = {-1};
-      final SqlValidatorScope[] ancestorScopes = {null};
-      SqlValidatorNamespace foundNs =
-          lookup.bb.scope.resolve(
-              originalRelName,
-              ancestorScopes,
-              nsIndexes);
-
-      assert foundNs != null;
-      assert nsIndexes.length == 1;
-
-      SqlValidatorScope ancestorScope = ancestorScopes[0];
-
-      // If the correlated reference is in a scope that's "above" the
-      // subquery, then this is a correlated subquery.
-      SqlValidatorScope parentScope = bb.scope;
-      do {
-        if (ancestorScope == parentScope) {
-          return false;
-        }
-        if (parentScope instanceof DelegatingScope) {
-          parentScope = ((DelegatingScope) parentScope).getParent();
-        } else {
-          break;
-        }
-      } while (parentScope != null);
-    }
-    return true;
-  }
-
-  /**
-   * Returns a list of fields to be prefixed to each relational expression.
-   *
-   * @return List of system fields
-   */
-  protected List<RelDataTypeField> getSystemFields() {
-    return Collections.emptyList();
-  }
-
-  private RexNode convertJoinCondition(
-      Blackboard bb,
-      SqlNode condition,
-      JoinConditionType conditionType,
-      RelNode leftRel,
-      RelNode rightRel) {
-    if (condition == null) {
-      return rexBuilder.makeLiteral(true);
-    }
-    bb.setRoot(ImmutableList.of(leftRel, rightRel));
-    replaceSubqueries(bb, condition, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
-    switch (conditionType) {
-    case ON:
-      bb.setRoot(ImmutableList.of(leftRel, rightRel));
-      return bb.convertExpression(condition);
-    case USING:
-      SqlNodeList list = (SqlNodeList) condition;
-      List<String> nameList = new ArrayList<String>();
-      for (SqlNode columnName : list) {
-        final SqlIdentifier id = (SqlIdentifier) columnName;
-        String name = id.getSimple();
-        nameList.add(name);
-      }
-      return convertUsing(leftRel, rightRel, nameList);
-    default:
-      throw Util.unexpected(conditionType);
-    }
-  }
-
-  /**
-   * Returns an expression for matching columns of a USING clause or inferred
-   * from NATURAL JOIN. "a JOIN b USING (x, y)" becomes "a.x = b.x AND a.y =
-   * b.y". Returns null if the column list is empty.
-   *
-   * @param leftRel  Left input to the join
-   * @param rightRel Right input to the join
-   * @param nameList List of column names to join on
-   * @return Expression to match columns from name list, or true if name list
-   * is empty
-   */
-  private RexNode convertUsing(
-      RelNode leftRel,
-      RelNode rightRel,
-      List<String> nameList) {
-    final List<RexNode> list = Lists.newArrayList();
-    for (String name : nameList) {
-      final RelDataType leftRowType = leftRel.getRowType();
-      RelDataTypeField leftField = catalogReader.field(leftRowType, name);
-      RexNode left =
-          rexBuilder.makeInputRef(
-              leftField.getType(),
-              leftField.getIndex());
-      final RelDataType rightRowType = rightRel.getRowType();
-      RelDataTypeField rightField =
-          catalogReader.field(rightRowType, name);
-      RexNode right =
-          rexBuilder.makeInputRef(
-              rightField.getType(),
-              leftRowType.getFieldList().size() + rightField.getIndex());
-      RexNode equalsCall =
-          rexBuilder.makeCall(
-              SqlStdOperatorTable.EQUALS,
-              left,
-              right);
-      list.add(equalsCall);
-    }
-    return RexUtil.composeConjunction(rexBuilder, list, false);
-  }
-
-  private static JoinRelType convertJoinType(JoinType joinType) {
-    switch (joinType) {
-    case COMMA:
-    case INNER:
-    case CROSS:
-      return JoinRelType.INNER;
-    case FULL:
-      return JoinRelType.FULL;
-    case LEFT:
-      return JoinRelType.LEFT;
-    case RIGHT:
-      return JoinRelType.RIGHT;
-    default:
-      throw Util.unexpected(joinType);
-    }
-  }
-
-  /**
-   * Converts the SELECT, GROUP BY and HAVING clauses of an aggregate query.
-   *
-   * <p>This method extracts SELECT, GROUP BY and HAVING clauses, and creates
-   * an {@link AggConverter}, then delegates to {@link #createAggImpl}.
-   * Derived class may override this method to change any of those clauses or
-   * specify a different {@link AggConverter}.
-   *
-   * @param bb            Scope within which to resolve identifiers
-   * @param select        Query
-   * @param orderExprList Additional expressions needed to implement ORDER BY
-   */
-  protected void convertAgg(
-      Blackboard bb,
-      SqlSelect select,
-      List<SqlNode> orderExprList) {
-    assert bb.root != null : "precondition: child != null";
-    SqlNodeList groupList = select.getGroup();
-    SqlNodeList selectList = select.getSelectList();
-    SqlNode having = select.getHaving();
-
-    final AggConverter aggConverter = new AggConverter(bb, select);
-    createAggImpl(
-        bb,
-        aggConverter,
-        selectList,
-        groupList,
-        having,
-        orderExprList);
-  }
-
-  protected final void createAggImpl(
-      Blackboard bb,
-      AggConverter aggConverter,
-      SqlNodeList selectList,
-      SqlNodeList groupList,
-      SqlNode having,
-      List<SqlNode> orderExprList) {
-    SqlNodeList aggList = new SqlNodeList(SqlParserPos.ZERO);
-
-    for (SqlNode selectNode : selectList) {
-      if (validator.isAggregate(selectNode)) {
-        aggList.add(selectNode);
-      }
-    }
-
-    // first replace the subqueries inside the aggregates
-    // because they will provide input rows to the aggregates.
-    replaceSubqueries(bb, aggList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
-
-    // If group-by clause is missing, pretend that it has zero elements.
-    if (groupList == null) {
-      groupList = SqlNodeList.EMPTY;
-    }
-
-    // register the group exprs
-
-    // build a map to remember the projections from the top scope to the
-    // output of the current root.
-    //
-    // Currently farrago allows expressions, not just column references in
-    // group by list. This is not SQL 2003 compliant.
-
-    Map<Integer, Integer> groupExprProjection =
-        new HashMap<Integer, Integer>();
-
-    int i = -1;
-    for (SqlNode groupExpr : groupList) {
-      ++i;
-      final SqlNode expandedGroupExpr =
-          validator.expand(groupExpr, bb.scope);
-      aggConverter.addGroupExpr(expandedGroupExpr);
-
-      if (expandedGroupExpr instanceof SqlIdentifier) {
-        // SQL 2003 does not allow expressions of column references
-        SqlIdentifier expr = (SqlIdentifier) expandedGroupExpr;
-
-        // column references should be fully qualified.
-        assert expr.names.size() == 2;
-        String originalRelName = expr.names.get(0);
-        String originalFieldName = expr.names.get(1);
-
-        int[] nsIndexes = {-1};
-        final SqlValidatorScope[] ancestorScopes = {null};
-        SqlValidatorNamespace foundNs =
-            bb.scope.resolve(
-                originalRelName,
-                ancestorScopes,
-                nsIndexes);
-
-        assert foundNs != null;
-        assert nsIndexes.length == 1;
-        int childNamespaceIndex = nsIndexes[0];
-
-        int namespaceOffset = 0;
-
-        if (childNamespaceIndex > 0) {
-          // If not the first child, need to figure out the width of
-          // output types from all the preceding namespaces
-          assert ancestorScopes[0] instanceof ListScope;
-          List<SqlValidatorNamespace> children =
-              ((ListScope) ancestorScopes[0]).getChildren();
-
-          for (int j = 0; j < childNamespaceIndex; j++) {
-            namespaceOffset +=
-                children.get(j).getRowType().getFieldCount();
-          }
-        }
-
-        RelDataTypeField field =
-            catalogReader.field(foundNs.getRowType(), originalFieldName);
-        int origPos = namespaceOffset + field.getIndex();
-
-        groupExprProjection.put(origPos, i);
-      }
-    }
-
-    RexNode havingExpr = null;
-    List<RexNode> selectExprs = new ArrayList<RexNode>();
-    List<String> selectNames = new ArrayList<String>();
-
-    try {
-      Util.permAssert(bb.agg == null, "already in agg mode");
-      bb.agg = aggConverter;
-
-      // convert the select and having expressions, so that the
-      // agg converter knows which aggregations are required
-
-      selectList.accept(aggConverter);
-      for (SqlNode expr : orderExprList) {
-        expr.accept(aggConverter);
-      }
-      if (having != null) {
-        having.accept(aggConverter);
-      }
-
-      // compute inputs to the aggregator
-      List<RexNode> preExprs = aggConverter.getPreExprs();
-      List<String> preNames = aggConverter.getPreNames();
-
-      if (preExprs.size() == 0) {
-        // Special case for COUNT(*), where we can end up with no inputs
-        // at all.  The rest of the system doesn't like 0-tuples, so we
-        // select a dummy constant here.
-        preExprs =
-            Collections.singletonList(
-                (RexNode) rexBuilder.makeExactLiteral(BigDecimal.ZERO));
-        preNames = Collections.singletonList(null);
-      }
-
-      final RelNode inputRel = bb.root;
-
-      // Project the expressions required by agg and having.
-      bb.setRoot(
-          RelOptUtil.createProject(
-              inputRel,
-              preExprs,
-              preNames,
-              true),
-          false);
-      bb.mapRootRelToFieldProjection.put(bb.root, groupExprProjection);
-
-      // REVIEW jvs 31-Oct-2007:  doesn't the declaration of
-      // monotonicity here assume sort-based aggregation at
-      // the physical level?
-
-      // Tell bb which of group columns are sorted.
-      bb.columnMonotonicities.clear();
-      for (SqlNode groupItem : groupList) {
-        bb.columnMonotonicities.add(
-            bb.scope.getMonotonicity(groupItem));
-      }
-
-      // Add the aggregator
-      bb.setRoot(
-          createAggregate(
-              bb,
-              BitSets.range(aggConverter.groupExprs.size()),
-              aggConverter.getAggCalls()),
-          false);
-
-      bb.mapRootRelToFieldProjection.put(bb.root, groupExprProjection);
-
-      // Replace subqueries in having here and modify having to use
-      // the replaced expressions
-      if (having

<TRUNCATED>


[04/13] incubator-kylin git commit: KYLIN-780 Update calcite 1.0.0 in pom

Posted by li...@apache.org.
KYLIN-780 Update calcite 1.0.0 in pom


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/650f81a3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/650f81a3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/650f81a3

Branch: refs/heads/KYLIN-780
Commit: 650f81a30f704ae2f5d3bd492f86c3bf89648966
Parents: 49ce391
Author: Yang Li <li...@apache.org>
Authored: Tue Jun 16 07:53:53 2015 +0800
Committer: Li, Yang <ya...@ebay.com>
Committed: Thu Jun 18 15:13:30 2015 +0800

----------------------------------------------------------------------
 jdbc/pom.xml  |  2 +-
 pom.xml       | 19 +++++++++----------
 query/pom.xml |  6 +++---
 3 files changed, 13 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/650f81a3/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/jdbc/pom.xml b/jdbc/pom.xml
index 7f707c7..485ed3d 100644
--- a/jdbc/pom.xml
+++ b/jdbc/pom.xml
@@ -22,7 +22,7 @@
 	<artifactId>kylin-jdbc</artifactId>
 	<packaging>jar</packaging>
 	<name>Kylin:JDBC</name>
-	<description>Kylin JDBC Driver on optiq avatica</description>
+	<description>Kylin JDBC Driver on Calcite Avatica</description>
 
 	<parent>
 		<groupId>org.apache.kylin</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/650f81a3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5748328..e49eb27 100644
--- a/pom.xml
+++ b/pom.xml
@@ -83,9 +83,8 @@
         <!-- REST Service -->
         <spring.framework.version>3.1.2.RELEASE</spring.framework.version>
 
-        <!-- Optiq Version -->
-        <optiq.version>0.9.2-incubating</optiq.version>
-        <linq4j.version>0.4</linq4j.version>
+        <!-- Calcite Version -->
+        <calcite.version>1.0.0-incubating</calcite.version>
 
         <!-- Metrics Codahale Version -->
         <metrics.version>3.0.1</metrics.version>
@@ -102,7 +101,7 @@
         <sonar.jacoco.reportPath>${project.basedir}/../target/jacoco.exec</sonar.jacoco.reportPath>
         <sonar.language>java</sonar.language>
         <sonar.jacoco.excludes>
-            org/apache/kylin/**/tools/**:**/*CLI.java:net/hydromatic/optiq/**:org/eigenbase/sql2rel/**
+            org/apache/kylin/**/tools/**:**/*CLI.java
         </sonar.jacoco.excludes>
 
     </properties>
@@ -248,21 +247,21 @@
                 <version>${yarn.version}</version>
             </dependency>
 
-            <!-- optiq dependencies -->
+            <!-- Calcite dependencies -->
             <dependency>
                 <groupId>org.apache.calcite</groupId>
                 <artifactId>calcite-core</artifactId>
-                <version>${optiq.version}</version>
+                <version>${calcite.version}</version>
             </dependency>
             <dependency>
                 <groupId>org.apache.calcite</groupId>
                 <artifactId>calcite-avatica</artifactId>
-                <version>${optiq.version}</version>
+                <version>${calcite.version}</version>
             </dependency>
             <dependency>
-                <groupId>net.hydromatic</groupId>
-                <artifactId>linq4j</artifactId>
-                <version>${linq4j.version}</version>
+                <groupId>org.apache.calcite</groupId>
+                <artifactId>calcite-linq4j</artifactId>
+                <version>${calcite.version}</version>
             </dependency>
 
             <!-- Other dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/650f81a3/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index 3881b4d..86fca54 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -23,7 +23,7 @@
     <artifactId>kylin-query</artifactId>
     <packaging>jar</packaging>
     <name>Kylin:Query</name>
-    <description>kylin query engine based on optiq</description>
+    <description>kylin query engine based on Calcite</description>
 
     <parent>
         <groupId>org.apache.kylin</groupId>
@@ -54,8 +54,8 @@
             <version>${project.parent.version}</version>
         </dependency>
         <dependency>
-            <groupId>net.hydromatic</groupId>
-            <artifactId>linq4j</artifactId>
+            <groupId>org.apache.calcite</groupId>
+            <artifactId>calcite-linq4j</artifactId>
         </dependency>
         <dependency>
             <groupId>log4j</groupId>


[08/13] incubator-kylin git commit: KYLIN-780, query compile pass

Posted by li...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/5d94f1aa/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
new file mode 100644
index 0000000..2508629
--- /dev/null
+++ b/atopcalcite/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -0,0 +1,4978 @@
+/*
+ * OVERRIDE POINT:
+ * - getInSubqueryThreshold(), was `20`, now `Integer.MAX_VALUE`
+ * - isTrimUnusedFields(), override to false
+ * - AggConverter.visit(SqlCall), skip column reading for COUNT(COL), for https://jirap.corp.ebay.com/browse/KYLIN-104
+ */
+
+/*
+ * 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.calcite.sql2rel;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptQuery;
+import org.apache.calcite.plan.RelOptSamplingParameters;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationImpl;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sample;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SemiJoinType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlInOperator;
+import org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.type.TableFunctionReturnTypeInference;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.AggregatingSelectScope;
+import org.apache.calcite.sql.validate.CollectNamespace;
+import org.apache.calcite.sql.validate.DelegatingScope;
+import org.apache.calcite.sql.validate.ListScope;
+import org.apache.calcite.sql.validate.ParameterScope;
+import org.apache.calcite.sql.validate.SelectScope;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlQualified;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.NumberUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.mapping.Mappings;
+import org.apache.calcite.util.trace.CalciteTrace;
+import org.apache.calcite.sql.SqlNumericLiteral;
+
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+import java.util.TreeSet;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import static org.apache.calcite.sql.SqlUtil.stripAs;
+import static org.apache.calcite.util.Static.RESOURCE;
+
+/**
+ * Converts a SQL parse tree (consisting of
+ * {@link org.apache.calcite.sql.SqlNode} objects) into a relational algebra
+ * expression (consisting of {@link org.apache.calcite.rel.RelNode} objects).
+ *
+ * <p>The public entry points are: {@link #convertQuery},
+ * {@link #convertExpression(SqlNode)}.
+ */
+public class SqlToRelConverter {
+  //~ Static fields/initializers ---------------------------------------------
+
+  protected static final Logger SQL2REL_LOGGER =
+      CalciteTrace.getSqlToRelTracer();
+
+  private static final Function<SubQuery, SqlNode> FN =
+      new Function<SubQuery, SqlNode>() {
+        public SqlNode apply(SubQuery input) {
+          return input.node;
+        }
+      };
+  private static final BigDecimal TWO = BigDecimal.valueOf(2L);
+
+  //~ Instance fields --------------------------------------------------------
+
+  protected final SqlValidator validator;
+  protected final RexBuilder rexBuilder;
+  protected final Prepare.CatalogReader catalogReader;
+  protected final RelOptCluster cluster;
+  private DefaultValueFactory defaultValueFactory;
+  private SubqueryConverter subqueryConverter;
+  protected final List<RelNode> leaves = new ArrayList<RelNode>();
+  private final List<SqlDynamicParam> dynamicParamSqlNodes =
+      new ArrayList<SqlDynamicParam>();
+  private final SqlOperatorTable opTab;
+  private boolean shouldConvertTableAccess;
+  protected final RelDataTypeFactory typeFactory;
+  private final SqlNodeToRexConverter exprConverter;
+  private boolean decorrelationEnabled;
+  private boolean trimUnusedFields;
+  private boolean shouldCreateValuesRel;
+  private boolean isExplain;
+  private int nDynamicParamsInExplain;
+
+  /**
+   * Fields used in name resolution for correlated subqueries.
+   */
+  private final Map<String, DeferredLookup> mapCorrelToDeferred =
+      new HashMap<String, DeferredLookup>();
+  private int nextCorrel = 0;
+
+  private static final String CORREL_PREFIX = "$cor";
+
+  /**
+   * Stack of names of datasets requested by the <code>
+   * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
+   */
+  private final Stack<String> datasetStack = new Stack<String>();
+
+  /**
+   * Mapping of non-correlated subqueries that have been converted to their
+   * equivalent constants. Used to avoid re-evaluating the subquery if it's
+   * already been evaluated.
+   */
+  private final Map<SqlNode, RexNode> mapConvertedNonCorrSubqs =
+      new HashMap<SqlNode, RexNode>();
+
+  public final RelOptTable.ViewExpander viewExpander;
+
+  //~ Constructors -----------------------------------------------------------
+  /**
+   * Creates a converter.
+   *
+   * @param viewExpander    Preparing statement
+   * @param validator       Validator
+   * @param catalogReader   Schema
+   * @param planner         Planner
+   * @param rexBuilder      Rex builder
+   * @param convertletTable Expression converter
+   */
+  public SqlToRelConverter(
+      RelOptTable.ViewExpander viewExpander,
+      SqlValidator validator,
+      Prepare.CatalogReader catalogReader,
+      RelOptPlanner planner,
+      RexBuilder rexBuilder,
+      SqlRexConvertletTable convertletTable) {
+    this.viewExpander = viewExpander;
+    this.opTab =
+        (validator
+            == null) ? SqlStdOperatorTable.instance()
+            : validator.getOperatorTable();
+    this.validator = validator;
+    this.catalogReader = catalogReader;
+    this.defaultValueFactory = new NullDefaultValueFactory();
+    this.subqueryConverter = new NoOpSubqueryConverter();
+    this.rexBuilder = rexBuilder;
+    this.typeFactory = rexBuilder.getTypeFactory();
+    RelOptQuery query = new RelOptQuery(planner);
+    this.cluster = query.createCluster(typeFactory, rexBuilder);
+    this.shouldConvertTableAccess = true;
+    this.exprConverter =
+        new SqlNodeToRexConverterImpl(convertletTable);
+    decorrelationEnabled = true;
+    trimUnusedFields = false;
+    shouldCreateValuesRel = true;
+    isExplain = false;
+    nDynamicParamsInExplain = 0;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  /**
+   * @return the RelOptCluster in use.
+   */
+  public RelOptCluster getCluster() {
+    return cluster;
+  }
+
+  /**
+   * Returns the row-expression builder.
+   */
+  public RexBuilder getRexBuilder() {
+    return rexBuilder;
+  }
+
+  /**
+   * Returns the number of dynamic parameters encountered during translation;
+   * this must only be called after {@link #convertQuery}.
+   *
+   * @return number of dynamic parameters
+   */
+  public int getDynamicParamCount() {
+    return dynamicParamSqlNodes.size();
+  }
+
+  /**
+   * Returns the type inferred for a dynamic parameter.
+   *
+   * @param index 0-based index of dynamic parameter
+   * @return inferred type, never null
+   */
+  public RelDataType getDynamicParamType(int index) {
+    SqlNode sqlNode = dynamicParamSqlNodes.get(index);
+    if (sqlNode == null) {
+      throw Util.needToImplement("dynamic param type inference");
+    }
+    return validator.getValidatedNodeType(sqlNode);
+  }
+
+  /**
+   * Returns the current count of the number of dynamic parameters in an
+   * EXPLAIN PLAN statement.
+   *
+   * @param increment if true, increment the count
+   * @return the current count before the optional increment
+   */
+  public int getDynamicParamCountInExplain(boolean increment) {
+    int retVal = nDynamicParamsInExplain;
+    if (increment) {
+      ++nDynamicParamsInExplain;
+    }
+    return retVal;
+  }
+
+  /**
+   * @return mapping of non-correlated subqueries that have been converted to
+   * the constants that they evaluate to
+   */
+  public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
+    return mapConvertedNonCorrSubqs;
+  }
+
+  /**
+   * Adds to the current map of non-correlated converted subqueries the
+   * elements from another map that contains non-correlated subqueries that
+   * have been converted by another SqlToRelConverter.
+   *
+   * @param alreadyConvertedNonCorrSubqs the other map
+   */
+  public void addConvertedNonCorrSubqs(
+      Map<SqlNode, RexNode> alreadyConvertedNonCorrSubqs) {
+    mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs);
+  }
+
+  /**
+   * Set a new DefaultValueFactory. To have any effect, this must be called
+   * before any convert method.
+   *
+   * @param factory new DefaultValueFactory
+   */
+  public void setDefaultValueFactory(DefaultValueFactory factory) {
+    defaultValueFactory = factory;
+  }
+
+  /**
+   * Sets a new SubqueryConverter. To have any effect, this must be called
+   * before any convert method.
+   *
+   * @param converter new SubqueryConverter
+   */
+  public void setSubqueryConverter(SubqueryConverter converter) {
+    subqueryConverter = converter;
+  }
+
+  /**
+   * Indicates that the current statement is part of an EXPLAIN PLAN statement
+   *
+   * @param nDynamicParams number of dynamic parameters in the statement
+   */
+  public void setIsExplain(int nDynamicParams) {
+    isExplain = true;
+    nDynamicParamsInExplain = nDynamicParams;
+  }
+
+  /**
+   * Controls whether table access references are converted to physical rels
+   * immediately. The optimizer doesn't like leaf rels to have
+   * {@link Convention#NONE}. However, if we are doing further conversion
+   * passes (e.g. {@link RelStructuredTypeFlattener}), then we may need to
+   * defer conversion. To have any effect, this must be called before any
+   * convert method.
+   *
+   * @param enabled true for immediate conversion (the default); false to
+   *                generate logical LogicalTableScan instances
+   */
+  public void enableTableAccessConversion(boolean enabled) {
+    shouldConvertTableAccess = enabled;
+  }
+
+  /**
+   * Controls whether instances of
+   * {@link org.apache.calcite.rel.logical.LogicalValues} are generated. These
+   * may not be supported by all physical implementations. To have any effect,
+   * this must be called before any convert method.
+   *
+   * @param enabled true to allow LogicalValues to be generated (the default);
+   *                false to force substitution of Project+OneRow instead
+   */
+  public void enableValuesRelCreation(boolean enabled) {
+    shouldCreateValuesRel = enabled;
+  }
+
+  private void checkConvertedType(SqlNode query, RelNode result) {
+    if (!query.isA(SqlKind.DML)) {
+      // Verify that conversion from SQL to relational algebra did
+      // not perturb any type information.  (We can't do this if the
+      // SQL statement is something like an INSERT which has no
+      // validator type information associated with its result,
+      // hence the namespace check above.)
+      RelDataType convertedRowType = result.getRowType();
+      if (!checkConvertedRowType(query, convertedRowType)) {
+        RelDataType validatedRowType =
+            validator.getValidatedNodeType(query);
+        validatedRowType = uniquifyFields(validatedRowType);
+        throw Util.newInternal("Conversion to relational algebra failed to "
+            + "preserve datatypes:\n"
+            + "validated type:\n"
+            + validatedRowType.getFullTypeString()
+            + "\nconverted type:\n"
+            + convertedRowType.getFullTypeString()
+            + "\nrel:\n"
+            + RelOptUtil.toString(result));
+      }
+    }
+  }
+
+  public RelNode flattenTypes(
+      RelNode rootRel,
+      boolean restructure) {
+    RelStructuredTypeFlattener typeFlattener =
+        new RelStructuredTypeFlattener(rexBuilder, createToRelContext());
+    return typeFlattener.rewrite(rootRel, restructure);
+  }
+
+  /**
+   * If subquery is correlated and decorrelation is enabled, performs
+   * decorrelation.
+   *
+   * @param query   Query
+   * @param rootRel Root relational expression
+   * @return New root relational expression after decorrelation
+   */
+  public RelNode decorrelate(SqlNode query, RelNode rootRel) {
+    if (!enableDecorrelation()) {
+      return rootRel;
+    }
+    final RelNode result = decorrelateQuery(rootRel);
+    if (result != rootRel) {
+      checkConvertedType(query, result);
+    }
+    return result;
+  }
+
+  /**
+   * Walks over a tree of relational expressions, replacing each
+   * {@link RelNode} with a 'slimmed down' relational expression that projects
+   * only the fields required by its consumer.
+   *
+   * <p>This may make things easier for the optimizer, by removing crud that
+   * would expand the search space, but is difficult for the optimizer itself
+   * to do it, because optimizer rules must preserve the number and type of
+   * fields. Hence, this transform that operates on the entire tree, similar
+   * to the {@link RelStructuredTypeFlattener type-flattening transform}.
+   *
+   * <p>Currently this functionality is disabled in farrago/luciddb; the
+   * default implementation of this method does nothing.
+   *
+   * @param rootRel Relational expression that is at the root of the tree
+   * @return Trimmed relational expression
+   */
+  public RelNode trimUnusedFields(RelNode rootRel) {
+    // Trim fields that are not used by their consumer.
+    if (isTrimUnusedFields()) {
+      final RelFieldTrimmer trimmer = newFieldTrimmer();
+      rootRel = trimmer.trim(rootRel);
+      boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
+      if (dumpPlan) {
+        SQL2REL_LOGGER.fine(
+            RelOptUtil.dumpPlan(
+                "Plan after trimming unused fields",
+                rootRel,
+                false,
+                SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+      }
+    }
+    return rootRel;
+  }
+
+  /**
+   * Creates a RelFieldTrimmer.
+   *
+   * @return Field trimmer
+   */
+  protected RelFieldTrimmer newFieldTrimmer() {
+    return new RelFieldTrimmer(validator);
+  }
+
+  /**
+   * Converts an unvalidated query's parse tree into a relational expression.
+   *
+   * @param query           Query to convert
+   * @param needsValidation Whether to validate the query before converting;
+   *                        <code>false</code> if the query has already been
+   *                        validated.
+   * @param top             Whether the query is top-level, say if its result
+   *                        will become a JDBC result set; <code>false</code> if
+   *                        the query will be part of a view.
+   */
+  public RelNode convertQuery(
+      SqlNode query,
+      final boolean needsValidation,
+      final boolean top) {
+    if (needsValidation) {
+      query = validator.validate(query);
+    }
+
+    RelNode result = convertQueryRecursive(query, top, null);
+    checkConvertedType(query, result);
+
+    boolean dumpPlan = SQL2REL_LOGGER.isLoggable(Level.FINE);
+    if (dumpPlan) {
+      SQL2REL_LOGGER.fine(
+          RelOptUtil.dumpPlan(
+              "Plan after converting SqlNode to RelNode",
+              result,
+              false,
+              SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+    }
+
+    return result;
+  }
+
+  protected boolean checkConvertedRowType(
+      SqlNode query,
+      RelDataType convertedRowType) {
+    RelDataType validatedRowType = validator.getValidatedNodeType(query);
+    validatedRowType = uniquifyFields(validatedRowType);
+
+    return RelOptUtil.equal(
+        "validated row type",
+        validatedRowType,
+        "converted row type",
+        convertedRowType,
+        false);
+  }
+
+  protected RelDataType uniquifyFields(RelDataType rowType) {
+    return validator.getTypeFactory().createStructType(
+        RelOptUtil.getFieldTypeList(rowType),
+        SqlValidatorUtil.uniquify(rowType.getFieldNames()));
+  }
+
+  /**
+   * Converts a SELECT statement's parse tree into a relational expression.
+   */
+  public RelNode convertSelect(SqlSelect select) {
+    final SqlValidatorScope selectScope = validator.getWhereScope(select);
+    final Blackboard bb = createBlackboard(selectScope, null);
+    convertSelectImpl(bb, select);
+    return bb.root;
+  }
+
+  /**
+   * Factory method for creating translation workspace.
+   */
+  protected Blackboard createBlackboard(
+      SqlValidatorScope scope,
+      Map<String, RexNode> nameToNodeMap) {
+    return new Blackboard(scope, nameToNodeMap);
+  }
+
+  /**
+   * Implementation of {@link #convertSelect(SqlSelect)}; derived class may
+   * override.
+   */
+  protected void convertSelectImpl(
+      final Blackboard bb,
+      SqlSelect select) {
+    convertFrom(
+        bb,
+        select.getFrom());
+    convertWhere(
+        bb,
+        select.getWhere());
+
+    List<SqlNode> orderExprList = new ArrayList<SqlNode>();
+    List<RelFieldCollation> collationList =
+        new ArrayList<RelFieldCollation>();
+    gatherOrderExprs(
+        bb,
+        select,
+        select.getOrderList(),
+        orderExprList,
+        collationList);
+    final RelCollation collation =
+        cluster.traitSetOf().canonize(RelCollationImpl.of(collationList));
+
+    if (validator.isAggregate(select)) {
+      convertAgg(
+          bb,
+          select,
+          orderExprList);
+    } else {
+      convertSelectList(
+          bb,
+          select,
+          orderExprList);
+    }
+
+    if (select.isDistinct()) {
+      distinctify(bb, true);
+    }
+    convertOrder(
+        select, bb, collation, orderExprList, select.getOffset(),
+        select.getFetch());
+    bb.setRoot(bb.root, true);
+  }
+
+  /**
+   * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds
+   * a relational expression to make the results unique.
+   *
+   * <p>If the SELECT clause contains duplicate expressions, adds
+   * {@link org.apache.calcite.rel.logical.LogicalProject}s so that we are
+   * grouping on the minimal set of keys. The performance gain isn't huge, but
+   * it is difficult to detect these duplicate expressions later.
+   *
+   * @param bb               Blackboard
+   * @param checkForDupExprs Check for duplicate expressions
+   */
+  private void distinctify(
+      Blackboard bb,
+      boolean checkForDupExprs) {
+    // Look for duplicate expressions in the project.
+    // Say we have 'select x, y, x, z'.
+    // Then dups will be {[2, 0]}
+    // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]}
+    RelNode rel = bb.root;
+    if (checkForDupExprs && (rel instanceof LogicalProject)) {
+      LogicalProject project = (LogicalProject) rel;
+      final List<RexNode> projectExprs = project.getProjects();
+      List<Integer> origins = new ArrayList<Integer>();
+      int dupCount = 0;
+      for (int i = 0; i < projectExprs.size(); i++) {
+        int x = findExpr(projectExprs.get(i), projectExprs, i);
+        if (x >= 0) {
+          origins.add(x);
+          ++dupCount;
+        } else {
+          origins.add(i);
+        }
+      }
+      if (dupCount == 0) {
+        distinctify(bb, false);
+        return;
+      }
+
+      final Map<Integer, Integer> squished = Maps.newHashMap();
+      final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+      final List<Pair<RexNode, String>> newProjects = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i++) {
+        if (origins.get(i) == i) {
+          squished.put(i, newProjects.size());
+          newProjects.add(RexInputRef.of2(i, fields));
+        }
+      }
+      rel =
+          new LogicalProject(
+              cluster,
+              rel,
+              Pair.left(newProjects),
+              Pair.right(newProjects),
+              LogicalProject.Flags.BOXED);
+
+      bb.root = rel;
+      distinctify(bb, false);
+      rel = bb.root;
+
+      // Create the expressions to reverse the mapping.
+      // Project($0, $1, $0, $2).
+      final List<Pair<RexNode, String>> undoProjects = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i++) {
+        final int origin = origins.get(i);
+        RelDataTypeField field = fields.get(i);
+        undoProjects.add(
+            Pair.of(
+                (RexNode) new RexInputRef(
+                    squished.get(origin), field.getType()),
+                field.getName()));
+      }
+
+      rel =
+          new LogicalProject(
+              cluster,
+              rel,
+              Pair.left(undoProjects),
+              Pair.right(undoProjects),
+              LogicalProject.Flags.BOXED);
+
+      bb.setRoot(
+          rel,
+          false);
+
+      return;
+    }
+
+    // Usual case: all of the expressions in the SELECT clause are
+    // different.
+    final ImmutableBitSet groupSet =
+        ImmutableBitSet.range(rel.getRowType().getFieldCount());
+    rel =
+        createAggregate(bb, false, groupSet, ImmutableList.of(groupSet),
+            ImmutableList.<AggregateCall>of());
+
+    bb.setRoot(
+        rel,
+        false);
+  }
+
+  private int findExpr(RexNode seek, List<RexNode> exprs, int count) {
+    for (int i = 0; i < count; i++) {
+      RexNode expr = exprs.get(i);
+      if (expr.toString().equals(seek.toString())) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  /**
+   * Converts a query's ORDER BY clause, if any.
+   *
+   * @param select        Query
+   * @param bb            Blackboard
+   * @param collation     Collation list
+   * @param orderExprList Method populates this list with orderBy expressions
+   *                      not present in selectList
+   * @param offset        Expression for number of rows to discard before
+   *                      returning first row
+   * @param fetch         Expression for number of rows to fetch
+   */
+  protected void convertOrder(
+      SqlSelect select,
+      Blackboard bb,
+      RelCollation collation,
+      List<SqlNode> orderExprList,
+      SqlNode offset,
+      SqlNode fetch) {
+    if (select.getOrderList() == null) {
+      assert collation.getFieldCollations().isEmpty();
+      if (offset == null && fetch == null) {
+        return;
+      }
+    }
+
+    // Create a sorter using the previously constructed collations.
+    bb.setRoot(
+        new Sort(
+            cluster,
+            cluster.traitSetOf(Convention.NONE, collation),
+            bb.root,
+            collation,
+            offset == null ? null : convertExpression(offset),
+            fetch == null ? null : convertExpression(fetch)),
+        false);
+
+    // If extra expressions were added to the project list for sorting,
+    // add another project to remove them.
+    if (orderExprList.size() > 0) {
+      List<RexNode> exprs = new ArrayList<RexNode>();
+      final RelDataType rowType = bb.root.getRowType();
+      final int fieldCount =
+          rowType.getFieldCount() - orderExprList.size();
+      for (int i = 0; i < fieldCount; i++) {
+        exprs.add(rexBuilder.makeInputRef(bb.root, i));
+      }
+      bb.setRoot(
+          new LogicalProject(
+              cluster,
+              cluster.traitSetOf(RelCollationImpl.PRESERVE),
+              bb.root,
+              exprs,
+              cluster.getTypeFactory().createStructType(
+                  rowType.getFieldList().subList(0, fieldCount)),
+              Project.Flags.BOXED),
+          false);
+    }
+  }
+
+  /**
+   * Returns whether a given node contains a {@link SqlInOperator}.
+   *
+   * @param node a RexNode tree
+   */
+  private static boolean containsInOperator(
+      SqlNode node) {
+    try {
+      SqlVisitor<Void> visitor =
+          new SqlBasicVisitor<Void>() {
+            public Void visit(SqlCall call) {
+              if (call.getOperator() instanceof SqlInOperator) {
+                throw new Util.FoundOne(call);
+              }
+              return super.visit(call);
+            }
+          };
+      node.accept(visitor);
+      return false;
+    } catch (Util.FoundOne e) {
+      Util.swallow(e, null);
+      return true;
+    }
+  }
+
+  /**
+   * Push down all the NOT logical operators into any IN/NOT IN operators.
+   *
+   * @param sqlNode the root node from which to look for NOT operators
+   * @return the transformed SqlNode representation with NOT pushed down.
+   */
+  private static SqlNode pushDownNotForIn(SqlNode sqlNode) {
+    if ((sqlNode instanceof SqlCall) && containsInOperator(sqlNode)) {
+      SqlCall sqlCall = (SqlCall) sqlNode;
+      if ((sqlCall.getOperator() == SqlStdOperatorTable.AND)
+          || (sqlCall.getOperator() == SqlStdOperatorTable.OR)) {
+        SqlNode[] sqlOperands = ((SqlBasicCall) sqlCall).operands;
+        for (int i = 0; i < sqlOperands.length; i++) {
+          sqlOperands[i] = pushDownNotForIn(sqlOperands[i]);
+        }
+        return sqlNode;
+      } else if (sqlCall.getOperator() == SqlStdOperatorTable.NOT) {
+        SqlNode childNode = sqlCall.operand(0);
+        assert childNode instanceof SqlCall;
+        SqlBasicCall childSqlCall = (SqlBasicCall) childNode;
+        if (childSqlCall.getOperator() == SqlStdOperatorTable.AND) {
+          SqlNode[] andOperands = childSqlCall.getOperands();
+          SqlNode[] orOperands = new SqlNode[andOperands.length];
+          for (int i = 0; i < orOperands.length; i++) {
+            orOperands[i] =
+                SqlStdOperatorTable.NOT.createCall(
+                    SqlParserPos.ZERO,
+                    andOperands[i]);
+          }
+          for (int i = 0; i < orOperands.length; i++) {
+            orOperands[i] = pushDownNotForIn(orOperands[i]);
+          }
+          return SqlStdOperatorTable.OR.createCall(SqlParserPos.ZERO,
+              orOperands[0], orOperands[1]);
+        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.OR) {
+          SqlNode[] orOperands = childSqlCall.getOperands();
+          SqlNode[] andOperands = new SqlNode[orOperands.length];
+          for (int i = 0; i < andOperands.length; i++) {
+            andOperands[i] =
+                SqlStdOperatorTable.NOT.createCall(
+                    SqlParserPos.ZERO,
+                    orOperands[i]);
+          }
+          for (int i = 0; i < andOperands.length; i++) {
+            andOperands[i] = pushDownNotForIn(andOperands[i]);
+          }
+          return SqlStdOperatorTable.AND.createCall(SqlParserPos.ZERO,
+              andOperands[0], andOperands[1]);
+        } else if (childSqlCall.getOperator() == SqlStdOperatorTable.NOT) {
+          SqlNode[] notOperands = childSqlCall.getOperands();
+          assert notOperands.length == 1;
+          return pushDownNotForIn(notOperands[0]);
+        } else if (childSqlCall.getOperator() instanceof SqlInOperator) {
+          SqlNode[] inOperands = childSqlCall.getOperands();
+          SqlInOperator inOp =
+              (SqlInOperator) childSqlCall.getOperator();
+          if (inOp.isNotIn()) {
+            return SqlStdOperatorTable.IN.createCall(
+                SqlParserPos.ZERO,
+                inOperands[0],
+                inOperands[1]);
+          } else {
+            return SqlStdOperatorTable.NOT_IN.createCall(
+                SqlParserPos.ZERO,
+                inOperands[0],
+                inOperands[1]);
+          }
+        } else {
+          // childSqlCall is "leaf" node in a logical expression tree
+          // (only considering AND, OR, NOT)
+          return sqlNode;
+        }
+      } else {
+        // sqlNode is "leaf" node in a logical expression tree
+        // (only considering AND, OR, NOT)
+        return sqlNode;
+      }
+    } else {
+      // tree rooted at sqlNode does not contain inOperator
+      return sqlNode;
+    }
+  }
+
+  /**
+   * Converts a WHERE clause.
+   *
+   * @param bb    Blackboard
+   * @param where WHERE clause, may be null
+   */
+  private void convertWhere(
+      final Blackboard bb,
+      final SqlNode where) {
+    if (where == null) {
+      return;
+    }
+    SqlNode newWhere = pushDownNotForIn(where);
+    replaceSubqueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+    final RexNode convertedWhere = bb.convertExpression(newWhere);
+
+    // only allocate filter if the condition is not TRUE
+    if (!convertedWhere.isAlwaysTrue()) {
+      bb.setRoot(
+          RelOptUtil.createFilter(bb.root, convertedWhere),
+          false);
+    }
+  }
+
+  private void replaceSubqueries(
+      final Blackboard bb,
+      final SqlNode expr,
+      RelOptUtil.Logic logic) {
+    findSubqueries(bb, expr, logic, false);
+    for (SubQuery node : bb.subqueryList) {
+      substituteSubquery(bb, node);
+    }
+  }
+
+  private void substituteSubquery(Blackboard bb, SubQuery subQuery) {
+    final RexNode expr = subQuery.expr;
+    if (expr != null) {
+      // Already done.
+      return;
+    }
+
+    final SqlBasicCall call;
+    final RelNode rel;
+    final SqlNode query;
+    final Pair<RelNode, Boolean> converted;
+    switch (subQuery.node.getKind()) {
+    case CURSOR:
+      convertCursor(bb, subQuery);
+      return;
+
+    case MULTISET_QUERY_CONSTRUCTOR:
+    case MULTISET_VALUE_CONSTRUCTOR:
+      rel = convertMultisets(ImmutableList.of(subQuery.node), bb);
+      subQuery.expr = bb.register(rel, JoinRelType.INNER);
+      return;
+
+    case IN:
+      call = (SqlBasicCall) subQuery.node;
+      final SqlNode[] operands = call.getOperands();
+
+      SqlNode leftKeyNode = operands[0];
+      query = operands[1];
+
+      final List<RexNode> leftKeys;
+      switch (leftKeyNode.getKind()) {
+      case ROW:
+        leftKeys = Lists.newArrayList();
+        for (SqlNode sqlExpr : ((SqlBasicCall) leftKeyNode).getOperandList()) {
+          leftKeys.add(bb.convertExpression(sqlExpr));
+        }
+        break;
+      default:
+        leftKeys = ImmutableList.of(bb.convertExpression(leftKeyNode));
+      }
+
+      final boolean isNotIn = ((SqlInOperator) call.getOperator()).isNotIn();
+      if (query instanceof SqlNodeList) {
+        SqlNodeList valueList = (SqlNodeList) query;
+        if (!containsNullLiteral(valueList)
+            && valueList.size() < getInSubqueryThreshold()) {
+          // We're under the threshold, so convert to OR.
+          subQuery.expr =
+              convertInToOr(
+                  bb,
+                  leftKeys,
+                  valueList,
+                  isNotIn);
+          return;
+        }
+
+        // Otherwise, let convertExists translate
+        // values list into an inline table for the
+        // reference to Q below.
+      }
+
+      // Project out the search columns from the left side
+
+      //  Q1:
+      // "select from emp where emp.deptno in (select col1 from T)"
+      //
+      // is converted to
+      //
+      // "select from
+      //   emp inner join (select distinct col1 from T)) q
+      //   on emp.deptno = q.col1
+      //
+      // Q2:
+      // "select from emp where emp.deptno not in (Q)"
+      //
+      // is converted to
+      //
+      // "select from
+      //   emp left outer join (select distinct col1, TRUE from T) q
+      //   on emp.deptno = q.col1
+      //   where emp.deptno <> null
+      //         and q.indicator <> TRUE"
+      //
+      final boolean outerJoin = bb.subqueryNeedsOuterJoin
+          || isNotIn
+          || subQuery.logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
+      converted =
+          convertExists(query, RelOptUtil.SubqueryType.IN, subQuery.logic,
+              outerJoin);
+      if (converted.right) {
+        // Generate
+        //    emp CROSS JOIN (SELECT COUNT(*) AS c,
+        //                       COUNT(deptno) AS ck FROM dept)
+        final RelDataType longType =
+            typeFactory.createSqlType(SqlTypeName.BIGINT);
+        final RelNode seek = converted.left.getInput(0); // fragile
+        final int keyCount = leftKeys.size();
+        final List<Integer> args = ImmutableIntList.range(0, keyCount);
+        LogicalAggregate aggregate =
+            new LogicalAggregate(cluster, seek, false, ImmutableBitSet.of(),
+                null,
+                ImmutableList.of(
+                    new AggregateCall(SqlStdOperatorTable.COUNT, false,
+                        ImmutableList.<Integer>of(), longType, null),
+                    new AggregateCall(SqlStdOperatorTable.COUNT, false,
+                        args, longType, null)));
+        LogicalJoin join =
+            new LogicalJoin(cluster, bb.root, aggregate,
+                rexBuilder.makeLiteral(true), JoinRelType.INNER,
+                ImmutableSet.<String>of());
+        bb.setRoot(join, false);
+      }
+      RexNode rex =
+          bb.register(converted.left,
+              outerJoin ? JoinRelType.LEFT : JoinRelType.INNER, leftKeys);
+
+      subQuery.expr = translateIn(subQuery, bb.root, rex);
+      if (isNotIn) {
+        subQuery.expr =
+            rexBuilder.makeCall(SqlStdOperatorTable.NOT, subQuery.expr);
+      }
+      return;
+
+    case EXISTS:
+      // "select from emp where exists (select a from T)"
+      //
+      // is converted to the following if the subquery is correlated:
+      //
+      // "select from emp left outer join (select AGG_TRUE() as indicator
+      // from T group by corr_var) q where q.indicator is true"
+      //
+      // If there is no correlation, the expression is replaced with a
+      // boolean indicating whether the subquery returned 0 or >= 1 row.
+      call = (SqlBasicCall) subQuery.node;
+      query = call.getOperands()[0];
+      converted = convertExists(query, RelOptUtil.SubqueryType.EXISTS,
+          subQuery.logic, true);
+      assert !converted.right;
+      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, true)) {
+        return;
+      }
+      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
+      return;
+
+    case SCALAR_QUERY:
+      // Convert the subquery.  If it's non-correlated, convert it
+      // to a constant expression.
+      call = (SqlBasicCall) subQuery.node;
+      query = call.getOperands()[0];
+      converted = convertExists(query, RelOptUtil.SubqueryType.SCALAR,
+          subQuery.logic, true);
+      assert !converted.right;
+      if (convertNonCorrelatedSubQuery(subQuery, bb, converted.left, false)) {
+        return;
+      }
+      rel = convertToSingleValueSubq(query, converted.left);
+      subQuery.expr = bb.register(rel, JoinRelType.LEFT);
+      return;
+
+    case SELECT:
+      // This is used when converting multiset queries:
+      //
+      // select * from unnest(select multiset[deptno] from emps);
+      //
+      converted = convertExists(subQuery.node, RelOptUtil.SubqueryType.SCALAR,
+          subQuery.logic, true);
+      assert !converted.right;
+      subQuery.expr = bb.register(converted.left, JoinRelType.LEFT);
+      return;
+
+    default:
+      throw Util.newInternal("unexpected kind of subquery :" + subQuery.node);
+    }
+  }
+
+  private RexNode translateIn(SubQuery subQuery, RelNode root,
+      final RexNode rex) {
+    switch (subQuery.logic) {
+    case TRUE:
+      return rexBuilder.makeLiteral(true);
+
+    case UNKNOWN_AS_FALSE:
+      assert rex instanceof RexRangeRef;
+      final int fieldCount = rex.getType().getFieldCount();
+      RexNode rexNode = rexBuilder.makeFieldAccess(rex, fieldCount - 1);
+      rexNode = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, rexNode);
+
+      // Then append the IS NOT NULL(leftKeysForIn).
+      //
+      // RexRangeRef contains the following fields:
+      //   leftKeysForIn,
+      //   rightKeysForIn (the original subquery select list),
+      //   nullIndicator
+      //
+      // The first two lists contain the same number of fields.
+      final int k = (fieldCount - 1) / 2;
+      for (int i = 0; i < k; i++) {
+        rexNode =
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.AND,
+                rexNode,
+                rexBuilder.makeCall(
+                    SqlStdOperatorTable.IS_NOT_NULL,
+                    rexBuilder.makeFieldAccess(rex, i)));
+      }
+      return rexNode;
+
+    case TRUE_FALSE_UNKNOWN:
+    case UNKNOWN_AS_TRUE:
+      // select e.deptno,
+      //   case
+      //   when ct.c = 0 then false
+      //   when dt.i is not null then true
+      //   when e.deptno is null then null
+      //   when ct.ck < ct.c then null
+      //   else false
+      //   end
+      // from e
+      // cross join (select count(*) as c, count(deptno) as ck from v) as ct
+      // left join (select distinct deptno, true as i from v) as dt
+      //   on e.deptno = dt.deptno
+      final Join join = (Join) root;
+      final Project left = (Project) join.getLeft();
+      final RelNode leftLeft = ((Join) left.getInput(0)).getLeft();
+      final int leftLeftCount = leftLeft.getRowType().getFieldCount();
+      final RelDataType nullableBooleanType =
+          typeFactory.createTypeWithNullability(
+              typeFactory.createSqlType(SqlTypeName.BOOLEAN), true);
+      final RelDataType longType =
+          typeFactory.createSqlType(SqlTypeName.BIGINT);
+      final RexNode cRef = rexBuilder.makeInputRef(root, leftLeftCount);
+      final RexNode ckRef = rexBuilder.makeInputRef(root, leftLeftCount + 1);
+      final RexNode iRef =
+          rexBuilder.makeInputRef(root, root.getRowType().getFieldCount() - 1);
+
+      final RexLiteral zero =
+          rexBuilder.makeExactLiteral(BigDecimal.ZERO, longType);
+      final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
+      final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
+      final RexNode unknownLiteral =
+          rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
+
+      final ImmutableList.Builder<RexNode> args = ImmutableList.builder();
+      args.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, cRef, zero),
+          falseLiteral,
+          rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, iRef),
+          trueLiteral);
+      final JoinInfo joinInfo = join.analyzeCondition();
+      for (int leftKey : joinInfo.leftKeys) {
+        final RexNode kRef = rexBuilder.makeInputRef(root, leftKey);
+        args.add(rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, kRef),
+            unknownLiteral);
+      }
+      args.add(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, ckRef, cRef),
+          unknownLiteral,
+          falseLiteral);
+
+      return rexBuilder.makeCall(
+          nullableBooleanType,
+          SqlStdOperatorTable.CASE,
+          args.build());
+
+    default:
+      throw new AssertionError(subQuery.logic);
+    }
+  }
+
+  private static boolean containsNullLiteral(SqlNodeList valueList) {
+    for (SqlNode node : valueList.getList()) {
+      if (node instanceof SqlLiteral) {
+        SqlLiteral lit = (SqlLiteral) node;
+        if (lit.getValue() == null) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Determines if a subquery is non-correlated and if so, converts it to a
+   * constant.
+   *
+   * @param subQuery  the call that references the subquery
+   * @param bb        blackboard used to convert the subquery
+   * @param converted RelNode tree corresponding to the subquery
+   * @param isExists  true if the subquery is part of an EXISTS expression
+   * @return if the subquery can be converted to a constant
+   */
+  private boolean convertNonCorrelatedSubQuery(
+      SubQuery subQuery,
+      Blackboard bb,
+      RelNode converted,
+      boolean isExists) {
+    SqlCall call = (SqlBasicCall) subQuery.node;
+    if (subqueryConverter.canConvertSubquery()
+        && isSubQueryNonCorrelated(converted, bb)) {
+      // First check if the subquery has already been converted
+      // because it's a nested subquery.  If so, don't re-evaluate
+      // it again.
+      RexNode constExpr = mapConvertedNonCorrSubqs.get(call);
+      if (constExpr == null) {
+        constExpr =
+            subqueryConverter.convertSubquery(
+                call,
+                this,
+                isExists,
+                isExplain);
+      }
+      if (constExpr != null) {
+        subQuery.expr = constExpr;
+        mapConvertedNonCorrSubqs.put(call, constExpr);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Converts the RelNode tree for a select statement to a select that
+   * produces a single value.
+   *
+   * @param query the query
+   * @param plan   the original RelNode tree corresponding to the statement
+   * @return the converted RelNode tree
+   */
+  public RelNode convertToSingleValueSubq(
+      SqlNode query,
+      RelNode plan) {
+    // Check whether query is guaranteed to produce a single value.
+    if (query instanceof SqlSelect) {
+      SqlSelect select = (SqlSelect) query;
+      SqlNodeList selectList = select.getSelectList();
+      SqlNodeList groupList = select.getGroup();
+
+      if ((selectList.size() == 1)
+          && ((groupList == null) || (groupList.size() == 0))) {
+        SqlNode selectExpr = selectList.get(0);
+        if (selectExpr instanceof SqlCall) {
+          SqlCall selectExprCall = (SqlCall) selectExpr;
+          if (selectExprCall.getOperator()
+              instanceof SqlAggFunction) {
+            return plan;
+          }
+        }
+      }
+    }
+
+    // If not, project SingleValueAgg
+    return RelOptUtil.createSingleValueAggRel(
+        cluster,
+        plan);
+  }
+
+  /**
+   * Converts "x IN (1, 2, ...)" to "x=1 OR x=2 OR ...".
+   *
+   * @param leftKeys   LHS
+   * @param valuesList RHS
+   * @param isNotIn    is this a NOT IN operator
+   * @return converted expression
+   */
+  private RexNode convertInToOr(
+      final Blackboard bb,
+      final List<RexNode> leftKeys,
+      SqlNodeList valuesList,
+      boolean isNotIn) {
+    List<RexNode> comparisons = new ArrayList<RexNode>();
+    for (SqlNode rightVals : valuesList) {
+      RexNode rexComparison;
+      if (leftKeys.size() == 1) {
+        rexComparison =
+            rexBuilder.makeCall(
+                SqlStdOperatorTable.EQUALS,
+                leftKeys.get(0),
+                bb.convertExpression(rightVals));
+      } else {
+        assert rightVals instanceof SqlCall;
+        final SqlBasicCall call = (SqlBasicCall) rightVals;
+        assert (call.getOperator() instanceof SqlRowOperator)
+            && call.getOperands().length == leftKeys.size();
+        rexComparison =
+            RexUtil.composeConjunction(
+                rexBuilder,
+                Iterables.transform(
+                    Pair.zip(leftKeys, call.getOperandList()),
+                    new Function<Pair<RexNode, SqlNode>, RexNode>() {
+                      public RexNode apply(Pair<RexNode, SqlNode> pair) {
+                        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                            pair.left, bb.convertExpression(pair.right));
+                      }
+                    }),
+                false);
+      }
+      comparisons.add(rexComparison);
+    }
+
+    RexNode result =
+        RexUtil.composeDisjunction(rexBuilder, comparisons, true);
+    assert result != null;
+
+    if (isNotIn) {
+      result =
+          rexBuilder.makeCall(
+              SqlStdOperatorTable.NOT,
+              result);
+    }
+
+    return result;
+  }
+
+  /**
+   * Gets the list size threshold under which {@link #convertInToOr} is used.
+   * Lists of this size or greater will instead be converted to use a join
+   * against an inline table
+   * ({@link org.apache.calcite.rel.logical.LogicalValues}) rather than a
+   * predicate. A threshold of 0 forces usage of an inline table in all cases; a
+   * threshold of Integer.MAX_VALUE forces usage of OR in all cases
+   *
+   * @return threshold, default 20
+   */
+  protected int getInSubqueryThreshold() {
+    // OVERRIDE POINT
+    return Integer.MAX_VALUE; // was 20
+  }
+
+  /**
+   * Converts an EXISTS or IN predicate into a join. For EXISTS, the subquery
+   * produces an indicator variable, and the result is a relational expression
+   * which outer joins that indicator to the original query. After performing
+   * the outer join, the condition will be TRUE if the EXISTS condition holds,
+   * NULL otherwise.
+   *
+   * @param seek           A query, for example 'select * from emp' or
+   *                       'values (1,2,3)' or '('Foo', 34)'.
+   * @param subqueryType   Whether sub-query is IN, EXISTS or scalar
+   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
+   *     FALSE, UNKNOWN) will be required, or whether we can accept an
+   *     approximation (say representing UNKNOWN as FALSE)
+   * @param needsOuterJoin Whether an outer join is needed
+   * @return join expression
+   * @pre extraExpr == null || extraName != null
+   */
+  private Pair<RelNode, Boolean> convertExists(
+      SqlNode seek,
+      RelOptUtil.SubqueryType subqueryType,
+      RelOptUtil.Logic logic,
+      boolean needsOuterJoin) {
+    final SqlValidatorScope seekScope =
+        (seek instanceof SqlSelect)
+            ? validator.getSelectScope((SqlSelect) seek)
+            : null;
+    final Blackboard seekBb = createBlackboard(seekScope, null);
+    RelNode seekRel = convertQueryOrInList(seekBb, seek);
+
+    return RelOptUtil.createExistsPlan(seekRel, subqueryType, logic,
+        needsOuterJoin);
+  }
+
+  private RelNode convertQueryOrInList(
+      Blackboard bb,
+      SqlNode seek) {
+    // NOTE: Once we start accepting single-row queries as row constructors,
+    // there will be an ambiguity here for a case like X IN ((SELECT Y FROM
+    // Z)).  The SQL standard resolves the ambiguity by saying that a lone
+    // select should be interpreted as a table expression, not a row
+    // expression.  The semantic difference is that a table expression can
+    // return multiple rows.
+    if (seek instanceof SqlNodeList) {
+      return convertRowValues(
+          bb,
+          seek,
+          ((SqlNodeList) seek).getList(),
+          false,
+          null);
+    } else {
+      return convertQueryRecursive(seek, false, null);
+    }
+  }
+
+  private RelNode convertRowValues(
+      Blackboard bb,
+      SqlNode rowList,
+      Collection<SqlNode> rows,
+      boolean allowLiteralsOnly,
+      RelDataType targetRowType) {
+    // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of
+    // literals into a single LogicalValues; this gives the optimizer a smaller
+    // input tree.  For everything else (computed expressions, row
+    // subqueries), we union each row in as a projection on top of a
+    // LogicalOneRow.
+
+    final ImmutableList.Builder<ImmutableList<RexLiteral>> tupleList =
+        ImmutableList.builder();
+    final RelDataType rowType;
+    if (targetRowType != null) {
+      rowType = targetRowType;
+    } else {
+      rowType =
+          SqlTypeUtil.promoteToRowType(
+              typeFactory,
+              validator.getValidatedNodeType(rowList),
+              null);
+    }
+
+    List<RelNode> unionInputs = new ArrayList<RelNode>();
+    for (SqlNode node : rows) {
+      SqlBasicCall call;
+      if (isRowConstructor(node)) {
+        call = (SqlBasicCall) node;
+        ImmutableList.Builder<RexLiteral> tuple = ImmutableList.builder();
+        for (Ord<SqlNode> operand : Ord.zip(call.operands)) {
+          RexLiteral rexLiteral =
+              convertLiteralInValuesList(
+                  operand.e,
+                  bb,
+                  rowType,
+                  operand.i);
+          if ((rexLiteral == null) && allowLiteralsOnly) {
+            return null;
+          }
+          if ((rexLiteral == null) || !shouldCreateValuesRel) {
+            // fallback to convertRowConstructor
+            tuple = null;
+            break;
+          }
+          tuple.add(rexLiteral);
+        }
+        if (tuple != null) {
+          tupleList.add(tuple.build());
+          continue;
+        }
+      } else {
+        RexLiteral rexLiteral =
+            convertLiteralInValuesList(
+                node,
+                bb,
+                rowType,
+                0);
+        if ((rexLiteral != null) && shouldCreateValuesRel) {
+          tupleList.add(ImmutableList.of(rexLiteral));
+          continue;
+        } else {
+          if ((rexLiteral == null) && allowLiteralsOnly) {
+            return null;
+          }
+        }
+
+        // convert "1" to "row(1)"
+        call =
+            (SqlBasicCall) SqlStdOperatorTable.ROW.createCall(
+                SqlParserPos.ZERO,
+                node);
+      }
+      unionInputs.add(convertRowConstructor(bb, call));
+    }
+    LogicalValues values =
+        new LogicalValues(
+            cluster,
+            rowType,
+            tupleList.build());
+    RelNode resultRel;
+    if (unionInputs.isEmpty()) {
+      resultRel = values;
+    } else {
+      if (!values.getTuples().isEmpty()) {
+        unionInputs.add(values);
+      }
+      LogicalUnion union =
+          new LogicalUnion(
+              cluster,
+              unionInputs,
+              true);
+      resultRel = union;
+    }
+    leaves.add(resultRel);
+    return resultRel;
+  }
+
+  private RexLiteral convertLiteralInValuesList(
+      SqlNode sqlNode,
+      Blackboard bb,
+      RelDataType rowType,
+      int iField) {
+    if (!(sqlNode instanceof SqlLiteral)) {
+      return null;
+    }
+    RelDataTypeField field = rowType.getFieldList().get(iField);
+    RelDataType type = field.getType();
+    if (type.isStruct()) {
+      // null literals for weird stuff like UDT's need
+      // special handling during type flattening, so
+      // don't use LogicalValues for those
+      return null;
+    }
+
+    RexNode literalExpr =
+        exprConverter.convertLiteral(
+            bb,
+            (SqlLiteral) sqlNode);
+
+    if (!(literalExpr instanceof RexLiteral)) {
+      assert literalExpr.isA(SqlKind.CAST);
+      RexNode child = ((RexCall) literalExpr).getOperands().get(0);
+      assert RexLiteral.isNullLiteral(child);
+
+      // NOTE jvs 22-Nov-2006:  we preserve type info
+      // in LogicalValues digest, so it's OK to lose it here
+      return (RexLiteral) child;
+    }
+
+    RexLiteral literal = (RexLiteral) literalExpr;
+
+    Comparable value = literal.getValue();
+
+    if (SqlTypeUtil.isExactNumeric(type)) {
+      BigDecimal roundedValue =
+          NumberUtil.rescaleBigDecimal(
+              (BigDecimal) value,
+              type.getScale());
+      return rexBuilder.makeExactLiteral(
+          roundedValue,
+          type);
+    }
+
+    if ((value instanceof NlsString)
+        && (type.getSqlTypeName() == SqlTypeName.CHAR)) {
+      // pad fixed character type
+      NlsString unpadded = (NlsString) value;
+      return rexBuilder.makeCharLiteral(
+          new NlsString(
+              Util.rpad(unpadded.getValue(), type.getPrecision()),
+              unpadded.getCharsetName(),
+              unpadded.getCollation()));
+    }
+    return literal;
+  }
+
+  private boolean isRowConstructor(SqlNode node) {
+    if (!(node.getKind() == SqlKind.ROW)) {
+      return false;
+    }
+    SqlCall call = (SqlCall) node;
+    return call.getOperator().getName().equalsIgnoreCase("row");
+  }
+
+  /**
+   * Builds a list of all <code>IN</code> or <code>EXISTS</code> operators
+   * inside SQL parse tree. Does not traverse inside queries.
+   *
+   * @param bb                           blackboard
+   * @param node                         the SQL parse tree
+   * @param logic Whether the answer needs to be in full 3-valued logic (TRUE,
+   *              FALSE, UNKNOWN) will be required, or whether we can accept
+   *              an approximation (say representing UNKNOWN as FALSE)
+   * @param registerOnlyScalarSubqueries if set to true and the parse tree
+   *                                     corresponds to a variation of a select
+   *                                     node, only register it if it's a scalar
+   *                                     subquery
+   */
+  private void findSubqueries(
+      Blackboard bb,
+      SqlNode node,
+      RelOptUtil.Logic logic,
+      boolean registerOnlyScalarSubqueries) {
+    final SqlKind kind = node.getKind();
+    switch (kind) {
+    case EXISTS:
+    case SELECT:
+    case MULTISET_QUERY_CONSTRUCTOR:
+    case MULTISET_VALUE_CONSTRUCTOR:
+    case CURSOR:
+    case SCALAR_QUERY:
+      if (!registerOnlyScalarSubqueries
+          || (kind == SqlKind.SCALAR_QUERY)) {
+        bb.registerSubquery(node, RelOptUtil.Logic.TRUE_FALSE);
+      }
+      return;
+    case IN:
+      if (((SqlCall) node).getOperator() == SqlStdOperatorTable.NOT_IN) {
+        logic = logic.negate();
+      }
+      break;
+    case NOT:
+      logic = logic.negate();
+      break;
+    }
+    if (node instanceof SqlCall) {
+      if (kind == SqlKind.OR
+          || kind == SqlKind.NOT) {
+        // It's always correct to outer join subquery with
+        // containing query; however, when predicates involve Or
+        // or NOT, outer join might be necessary.
+        bb.subqueryNeedsOuterJoin = true;
+      }
+      for (SqlNode operand : ((SqlCall) node).getOperandList()) {
+        if (operand != null) {
+          // In the case of an IN expression, locate scalar
+          // subqueries so we can convert them to constants
+          findSubqueries(
+              bb,
+              operand,
+              logic,
+              kind == SqlKind.IN || registerOnlyScalarSubqueries);
+        }
+      }
+    } else if (node instanceof SqlNodeList) {
+      for (SqlNode child : (SqlNodeList) node) {
+        findSubqueries(
+            bb,
+            child,
+            logic,
+            kind == SqlKind.IN || registerOnlyScalarSubqueries);
+      }
+    }
+
+    // Now that we've located any scalar subqueries inside the IN
+    // expression, register the IN expression itself.  We need to
+    // register the scalar subqueries first so they can be converted
+    // before the IN expression is converted.
+    if (kind == SqlKind.IN) {
+      if (logic == RelOptUtil.Logic.TRUE_FALSE_UNKNOWN
+          && !validator.getValidatedNodeType(node).isNullable()) {
+        logic = RelOptUtil.Logic.UNKNOWN_AS_FALSE;
+      }
+      // TODO: This conversion is only valid in the WHERE clause
+      if (logic == RelOptUtil.Logic.UNKNOWN_AS_FALSE
+          && !bb.subqueryNeedsOuterJoin) {
+        logic = RelOptUtil.Logic.TRUE;
+      }
+      bb.registerSubquery(node, logic);
+    }
+  }
+
+  /**
+   * Converts an expression from {@link SqlNode} to {@link RexNode} format.
+   *
+   * @param node Expression to translate
+   * @return Converted expression
+   */
+  public RexNode convertExpression(
+      SqlNode node) {
+    Map<String, RelDataType> nameToTypeMap = Collections.emptyMap();
+    Blackboard bb =
+        createBlackboard(
+            new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap),
+            null);
+    return bb.convertExpression(node);
+  }
+
+  /**
+   * Converts an expression from {@link SqlNode} to {@link RexNode} format,
+   * mapping identifier references to predefined expressions.
+   *
+   * @param node          Expression to translate
+   * @param nameToNodeMap map from String to {@link RexNode}; when an
+   *                      {@link SqlIdentifier} is encountered, it is used as a
+   *                      key and translated to the corresponding value from
+   *                      this map
+   * @return Converted expression
+   */
+  public RexNode convertExpression(
+      SqlNode node,
+      Map<String, RexNode> nameToNodeMap) {
+    final Map<String, RelDataType> nameToTypeMap =
+        new HashMap<String, RelDataType>();
+    for (Map.Entry<String, RexNode> entry : nameToNodeMap.entrySet()) {
+      nameToTypeMap.put(entry.getKey(), entry.getValue().getType());
+    }
+    Blackboard bb =
+        createBlackboard(
+            new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap),
+            nameToNodeMap);
+    return bb.convertExpression(node);
+  }
+
+  /**
+   * Converts a non-standard expression.
+   *
+   * <p>This method is an extension-point that derived classes can override. If
+   * this method returns a null result, the normal expression translation
+   * process will proceed. The default implementation always returns null.
+   *
+   * @param node Expression
+   * @param bb   Blackboard
+   * @return null to proceed with the usual expression translation process
+   */
+  protected RexNode convertExtendedExpression(
+      SqlNode node,
+      Blackboard bb) {
+    return null;
+  }
+
+  private RexNode convertOver(Blackboard bb, SqlNode node) {
+    SqlCall call = (SqlCall) node;
+    SqlCall aggCall = call.operand(0);
+    SqlNode windowOrRef = call.operand(1);
+    final SqlWindow window =
+        validator.resolveWindow(windowOrRef, bb.scope, true);
+    final SqlNodeList partitionList = window.getPartitionList();
+    final ImmutableList.Builder<RexNode> partitionKeys =
+        ImmutableList.builder();
+    for (SqlNode partition : partitionList) {
+      partitionKeys.add(bb.convertExpression(partition));
+    }
+    RexNode lowerBound = bb.convertExpression(window.getLowerBound());
+    RexNode upperBound = bb.convertExpression(window.getUpperBound());
+    SqlNodeList orderList = window.getOrderList();
+    if ((orderList.size() == 0) && !window.isRows()) {
+      // A logical range requires an ORDER BY clause. Use the implicit
+      // ordering of this relation. There must be one, otherwise it would
+      // have failed validation.
+      orderList = bb.scope.getOrderList();
+      if (orderList == null) {
+        throw new AssertionError(
+            "Relation should have sort key for implicit ORDER BY");
+      }
+    }
+    final ImmutableList.Builder<RexFieldCollation> orderKeys =
+        ImmutableList.builder();
+    final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
+    for (SqlNode order : orderList) {
+      flags.clear();
+      RexNode e = bb.convertSortExpression(order, flags);
+      orderKeys.add(new RexFieldCollation(e, flags));
+    }
+    try {
+      Util.permAssert(bb.window == null, "already in window agg mode");
+      bb.window = window;
+      RexNode rexAgg = exprConverter.convertCall(bb, aggCall);
+      rexAgg =
+          rexBuilder.ensureType(
+              validator.getValidatedNodeType(call), rexAgg, false);
+
+      // Walk over the tree and apply 'over' to all agg functions. This is
+      // necessary because the returned expression is not necessarily a call
+      // to an agg function. For example, AVG(x) becomes SUM(x) / COUNT(x).
+      final RexShuttle visitor =
+          new HistogramShuttle(
+              partitionKeys.build(), orderKeys.build(),
+              RexWindowBound.create(window.getLowerBound(), lowerBound),
+              RexWindowBound.create(window.getUpperBound(), upperBound),
+              window);
+      return rexAgg.accept(visitor);
+    } finally {
+      bb.window = null;
+    }
+  }
+
+  /**
+   * Converts a FROM clause into a relational expression.
+   *
+   * @param bb   Scope within which to resolve identifiers
+   * @param from FROM clause of a query. Examples include:
+   *
+   *             <ul>
+   *             <li>a single table ("SALES.EMP"),
+   *             <li>an aliased table ("EMP AS E"),
+   *             <li>a list of tables ("EMP, DEPT"),
+   *             <li>an ANSI Join expression ("EMP JOIN DEPT ON EMP.DEPTNO =
+   *             DEPT.DEPTNO"),
+   *             <li>a VALUES clause ("VALUES ('Fred', 20)"),
+   *             <li>a query ("(SELECT * FROM EMP WHERE GENDER = 'F')"),
+   *             <li>or any combination of the above.
+   *             </ul>
+   */
+  protected void convertFrom(
+      Blackboard bb,
+      SqlNode from) {
+    SqlCall call;
+    final SqlNode[] operands;
+    switch (from.getKind()) {
+    case AS:
+      operands = ((SqlBasicCall) from).getOperands();
+      convertFrom(bb, operands[0]);
+      return;
+
+    case WITH_ITEM:
+      convertFrom(bb, ((SqlWithItem) from).query);
+      return;
+
+    case WITH:
+      convertFrom(bb, ((SqlWith) from).body);
+      return;
+
+    case TABLESAMPLE:
+      operands = ((SqlBasicCall) from).getOperands();
+      SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands[1]);
+      if (sampleSpec instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) {
+        String sampleName =
+            ((SqlSampleSpec.SqlSubstitutionSampleSpec) sampleSpec)
+                .getName();
+        datasetStack.push(sampleName);
+        convertFrom(bb, operands[0]);
+        datasetStack.pop();
+      } else if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) {
+        SqlSampleSpec.SqlTableSampleSpec tableSampleSpec =
+            (SqlSampleSpec.SqlTableSampleSpec) sampleSpec;
+        convertFrom(bb, operands[0]);
+        RelOptSamplingParameters params =
+            new RelOptSamplingParameters(
+                tableSampleSpec.isBernoulli(),
+                tableSampleSpec.getSamplePercentage(),
+                tableSampleSpec.isRepeatable(),
+                tableSampleSpec.getRepeatableSeed());
+        bb.setRoot(new Sample(cluster, bb.root, params), false);
+      } else {
+        throw Util.newInternal(
+            "unknown TABLESAMPLE type: " + sampleSpec);
+      }
+      return;
+
+    case IDENTIFIER:
+      final SqlValidatorNamespace fromNamespace =
+          validator.getNamespace(from).resolve();
+      if (fromNamespace.getNode() != null) {
+        convertFrom(bb, fromNamespace.getNode());
+        return;
+      }
+      final String datasetName =
+          datasetStack.isEmpty() ? null : datasetStack.peek();
+      boolean[] usedDataset = {false};
+      RelOptTable table =
+          SqlValidatorUtil.getRelOptTable(
+              fromNamespace,
+              catalogReader,
+              datasetName,
+              usedDataset);
+      final RelNode tableRel;
+      if (shouldConvertTableAccess) {
+        tableRel = toRel(table);
+      } else {
+        tableRel = new LogicalTableScan(cluster, table);
+      }
+      bb.setRoot(tableRel, true);
+      if (usedDataset[0]) {
+        bb.setDataset(datasetName);
+      }
+      return;
+
+    case JOIN:
+      final SqlJoin join = (SqlJoin) from;
+      final Blackboard fromBlackboard =
+          createBlackboard(validator.getJoinScope(from), null);
+      SqlNode left = join.getLeft();
+      SqlNode right = join.getRight();
+      final boolean isNatural = join.isNatural();
+      final JoinType joinType = join.getJoinType();
+      final Blackboard leftBlackboard =
+          createBlackboard(
+              Util.first(validator.getJoinScope(left),
+                  ((DelegatingScope) bb.scope).getParent()), null);
+      final Blackboard rightBlackboard =
+          createBlackboard(
+              Util.first(validator.getJoinScope(right),
+                  ((DelegatingScope) bb.scope).getParent()), null);
+      convertFrom(leftBlackboard, left);
+      RelNode leftRel = leftBlackboard.root;
+      convertFrom(rightBlackboard, right);
+      RelNode rightRel = rightBlackboard.root;
+      JoinRelType convertedJoinType = convertJoinType(joinType);
+      RexNode conditionExp;
+      if (isNatural) {
+        final List<String> columnList =
+            SqlValidatorUtil.deriveNaturalJoinColumnList(
+                validator.getNamespace(left).getRowType(),
+                validator.getNamespace(right).getRowType());
+        conditionExp = convertUsing(leftRel, rightRel, columnList);
+      } else {
+        conditionExp =
+            convertJoinCondition(
+                fromBlackboard,
+                join.getCondition(),
+                join.getConditionType(),
+                leftRel,
+                rightRel);
+      }
+
+      final RelNode joinRel =
+          createJoin(
+              fromBlackboard,
+              leftRel,
+              rightRel,
+              conditionExp,
+              convertedJoinType);
+      bb.setRoot(joinRel, false);
+      return;
+
+    case SELECT:
+    case INTERSECT:
+    case EXCEPT:
+    case UNION:
+      final RelNode rel = convertQueryRecursive(from, false, null);
+      bb.setRoot(rel, true);
+      return;
+
+    case VALUES:
+      convertValuesImpl(bb, (SqlCall) from, null);
+      return;
+
+    case UNNEST:
+      final SqlNode node = ((SqlCall) from).operand(0);
+      replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+      final RelNode childRel =
+          RelOptUtil.createProject(
+              (null != bb.root) ? bb.root : LogicalValues.createOneRow(cluster),
+              Collections.singletonList(bb.convertExpression(node)),
+              Collections.singletonList(validator.deriveAlias(node, 0)),
+              true);
+
+      Uncollect uncollect =
+          new Uncollect(cluster, cluster.traitSetOf(Convention.NONE),
+              childRel);
+      bb.setRoot(uncollect, true);
+      return;
+
+    case COLLECTION_TABLE:
+      call = (SqlCall) from;
+
+      // Dig out real call; TABLE() wrapper is just syntactic.
+      assert call.getOperandList().size() == 1;
+      call = call.operand(0);
+      convertCollectionTable(bb, call);
+      return;
+
+    default:
+      throw Util.newInternal("not a join operator " + from);
+    }
+  }
+
+  protected void convertCollectionTable(
+      Blackboard bb,
+      SqlCall call) {
+    final SqlOperator operator = call.getOperator();
+    if (operator == SqlStdOperatorTable.TABLESAMPLE) {
+      final String sampleName =
+          SqlLiteral.stringValue(call.operand(0));
+      datasetStack.push(sampleName);
+      SqlCall cursorCall = call.operand(1);
+      SqlNode query = cursorCall.operand(0);
+      RelNode converted = convertQuery(query, false, false);
+      bb.setRoot(converted, false);
+      datasetStack.pop();
+      return;
+    }
+    replaceSubqueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+    // Expand table macro if possible. It's more efficient than
+    // LogicalTableFunctionScan.
+    if (operator instanceof SqlUserDefinedTableMacro) {
+      final SqlUserDefinedTableMacro udf =
+          (SqlUserDefinedTableMacro) operator;
+      final TranslatableTable table = udf.getTable(typeFactory,
+        call.getOperandList());
+      final RelDataType rowType = table.getRowType(typeFactory);
+      RelOptTable relOptTable = RelOptTableImpl.create(null, rowType, table);
+      RelNode converted = toRel(relOptTable);
+      bb.setRoot(converted, true);
+      return;
+    }
+
+    Type elementType;
+    if (operator instanceof SqlUserDefinedTableFunction) {
+      SqlUserDefinedTableFunction udtf = (SqlUserDefinedTableFunction) operator;
+      elementType = udtf.getElementType(typeFactory, call.getOperandList());
+    } else {
+      elementType = null;
+    }
+
+    RexNode rexCall = bb.convertExpression(call);
+    final List<RelNode> inputs = bb.retrieveCursors();
+    Set<RelColumnMapping> columnMappings =
+        getColumnMappings(operator);
+    LogicalTableFunctionScan callRel =
+        new LogicalTableFunctionScan(
+            cluster,
+            inputs,
+            rexCall,
+            elementType,
+            validator.getValidatedNodeType(call),
+            columnMappings);
+    bb.setRoot(callRel, true);
+    afterTableFunction(bb, call, callRel);
+  }
+
+  protected void afterTableFunction(
+      SqlToRelConverter.Blackboard bb,
+      SqlCall call,
+      LogicalTableFunctionScan callRel) {
+  }
+
+  private Set<RelColumnMapping> getColumnMappings(SqlOperator op) {
+    SqlReturnTypeInference rti = op.getReturnTypeInference();
+    if (rti == null) {
+      return null;
+    }
+    if (rti instanceof TableFunctionReturnTypeInference) {
+      TableFunctionReturnTypeInference tfrti =
+          (TableFunctionReturnTypeInference) rti;
+      return tfrti.getColumnMappings();
+    } else {
+      return null;
+    }
+  }
+
+  protected RelNode createJoin(
+      Blackboard bb,
+      RelNode leftRel,
+      RelNode rightRel,
+      RexNode joinCond,
+      JoinRelType joinType) {
+    assert joinCond != null;
+
+    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(rightRel);
+    if (correlatedVariables.size() > 0) {
+      final ImmutableBitSet.Builder requiredColumns = ImmutableBitSet.builder();
+      final List<String> correlNames = Lists.newArrayList();
+
+      // All correlations must refer the same namespace since correlation
+      // produces exactly one correlation source.
+      // The same source might be referenced by different variables since
+      // DeferredLookups are not de-duplicated at create time.
+      SqlValidatorNamespace prevNs = null;
+
+      for (String correlName : correlatedVariables) {
+        DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
+        RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName);
+        String originalRelName = lookup.getOriginalRelName();
+        String originalFieldName = fieldAccess.getField().getName();
+
+        int[] nsIndexes = {-1};
+        final SqlValidatorScope[] ancestorScopes = {null};
+        SqlValidatorNamespace foundNs =
+            lookup.bb.scope.resolve(
+                ImmutableList.of(originalRelName),
+                ancestorScopes,
+                nsIndexes);
+
+        assert foundNs != null;
+        assert nsIndexes.length == 1;
+
+        int childNamespaceIndex = nsIndexes[0];
+
+        SqlValidatorScope ancestorScope = ancestorScopes[0];
+        boolean correlInCurrentScope = ancestorScope == bb.scope;
+
+        if (!correlInCurrentScope) {
+          continue;
+        }
+
+        if (prevNs == null) {
+          prevNs = foundNs;
+        } else {
+          assert prevNs == foundNs : "All correlation variables should resolve"
+              + " to the same namespace."
+              + " Prev ns=" + prevNs
+              + ", new ns=" + foundNs;
+        }
+
+        int namespaceOffset = 0;
+        if (childNamespaceIndex > 0) {
+          // If not the first child, need to figure out the width
+          // of output types from all the preceding namespaces
+          assert ancestorScope instanceof ListScope;
+          List<SqlValidatorNamespace> children =
+              ((ListScope) ancestorScope).getChildren();
+
+          for (int i = 0; i < childNamespaceIndex; i++) {
+            SqlValidatorNamespace child = children.get(i);
+            namespaceOffset +=
+                child.getRowType().getFieldCount();
+          }
+        }
+
+        RelDataTypeField field =
+            catalogReader.field(foundNs.getRowType(), originalFieldName);
+        int pos = namespaceOffset + field.getIndex();
+
+        assert field.getType()
+            == lookup.getFieldAccess(correlName).getField().getType();
+
+        assert pos != -1;
+
+        if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) {
+          // bb.root is an aggregate and only projects group by
+          // keys.
+          Map<Integer, Integer> exprProjection =
+              bb.mapRootRelToFieldProjection.get(bb.root);
+
+          // subquery can reference group by keys projected from
+          // the root of the outer relation.
+          if (exprProjection.containsKey(pos)) {
+            pos = exprProjection.get(pos);
+          } else {
+            // correl not grouped
+            throw Util.newInternal(
+                "Identifier '" + originalRelName + "."
+                + originalFieldName + "' is not a group expr");
+          }
+        }
+
+        requiredColumns.set(pos);
+        correlNames.add(correlName);
+      }
+
+      if (!correlNames.isEmpty()) {
+        if (correlNames.size() > 1) {
+          // The same table was referenced more than once.
+          // So we deduplicate
+          RelShuttle dedup =
+              new DeduplicateCorrelateVariables(rexBuilder,
+                  correlNames.get(0),
+                  ImmutableSet.copyOf(Util.skip(correlNames)));
+          rightRel = rightRel.accept(dedup);
+        }
+        LogicalCorrelate corr = new LogicalCorrelate(
+            rightRel.getCluster(),
+            leftRel,
+            rightRel,
+            new CorrelationId(correlNames.get(0)),
+            requiredColumns.build(),
+            SemiJoinType.of(joinType));
+        if (!joinCond.isAlwaysTrue()) {
+          return RelOptUtil.createFilter(corr, joinCond);
+        }
+        return corr;
+      }
+    }
+
+    final List<RexNode> extraLeftExprs = new ArrayList<RexNode>();
+    final List<RexNode> extraRightExprs = new ArrayList<RexNode>();
+    final int leftCount = leftRel.getRowType().getFieldCount();
+    final int rightCount = rightRel.getRowType().getFieldCount();
+    if (!containsGet(joinCond)) {
+      joinCond = pushDownJoinConditions(
+          joinCond, leftCount, rightCount, extraLeftExprs, extraRightExprs);
+    }
+    if (!extraLeftExprs.isEmpty()) {
+      final List<RelDataTypeField> fields =
+          leftRel.getRowType().getFieldList();
+      leftRel = RelOptUtil.createProject(
+          leftRel,
+          new AbstractList<Pair<RexNode, String>>() {
+            @Override public int size() {
+              return leftCount + extraLeftExprs.size();
+            }
+
+            @Override public Pair<RexNode, String> get(int index) {
+              if (index < leftCount) {
+                RelDataTypeField field = fields.get(index);
+                return Pair.<RexNode, String>of(
+                    new RexInputRef(index, field.getType()),
+                    field.getName());
+              } else {
+                return Pair.<RexNode, String>of(
+                    extraLeftExprs.get(index - leftCount), null);
+              }
+            }
+          },
+          true);
+    }
+    if (!extraRightExprs.isEmpty()) {
+      final List<RelDataTypeField> fields =
+          rightRel.getRowType().getFieldList();
+      final int newLeftCount = leftCount + extraLeftExprs.size();
+      rightRel = RelOptUtil.createProject(
+          rightRel,
+          new AbstractList<Pair<RexNode, String>>() {
+            @Override public int size() {
+              return rightCount + extraRightExprs.size();
+            }
+
+            @Override public Pair<RexNode, String> get(int index) {
+              if (index < rightCount) {
+                RelDataTypeField field = fields.get(index);
+                return Pair.<RexNode, String>of(
+                    new RexInputRef(index, field.getType()),
+                    field.getName());
+              } else {
+                return Pair.of(
+                    RexUtil.shift(
+                        extraRightExprs.get(index - rightCount),
+                        -newLeftCount),
+                    null);
+              }
+            }
+          },
+          true);
+    }
+    RelNode join = createJoin(
+        leftRel,
+        rightRel,
+        joinCond,
+        joinType,
+        ImmutableSet.<String>of());
+    if (!extraLeftExprs.isEmpty() || !extraRightExprs.isEmpty()) {
+      Mappings.TargetMapping mapping =
+          Mappings.createShiftMapping(
+              leftCount + extraLeftExprs.size()
+                  + rightCount + extraRightExprs.size(),
+              0, 0, leftCount,
+              leftCount, leftCount + extraLeftExprs.size(), rightCount);
+      return RelOptUtil.project(join, mapping);
+    }
+    return join;
+  }
+
+  private static boolean containsGet(RexNode node) {
+    try {
+      node.accept(
+          new RexVisitorImpl<Void>(true) {
+            @Override public Void visitCall(RexCall call) {
+              if (call.getOperator() == RexBuilder.GET_OPERATOR) {
+                throw Util.FoundOne.NULL;
+              }
+              return super.visitCall(call);
+            }
+          });
+      return false;
+    } catch (Util.FoundOne e) {
+      return true;
+    }
+  }
+
+  /**
+   * Pushes down parts of a join condition. For example, given
+   * "emp JOIN dept ON emp.deptno + 1 = dept.deptno", adds a project above
+   * "emp" that computes the expression
+   * "emp.deptno + 1". The resulting join condition is a simple combination
+   * of AND, equals, and input fields.
+   */
+  private RexNode pushDownJoinConditions(
+      RexNode node,
+      int leftCount,
+      int rightCount,
+      List<RexNode> extraLeftExprs,
+      List<RexNode> extraRightExprs) {
+    switch (node.getKind()) {
+    case AND:
+    case OR:
+    case EQUALS:
+      RexCall call = (RexCall) node;
+      List<RexNode> list = new ArrayList<RexNode>();
+      List<RexNode> operands = Lists.newArrayList(call.getOperands());
+      for (int i = 0; i < operands.size(); i++) {
+        RexNode operand = operands.get(i);
+        final int left2 = leftCount + extraLeftExprs.size();
+        final int right2 = rightCount + extraRightExprs.size();
+        final RexNode e =
+            pushDownJoinConditions(
+                operand,
+                leftCount,
+                rightCount,
+                extraLeftExprs,
+                extraRightExprs);
+        final List<RexNode> remainingOperands = Util.skip(operands, i + 1);
+        final int left3 = leftCount + extraLeftExprs.size();
+        final int right3 = rightCount + extraRightExprs.size();
+        fix(remainingOperands, left2, left3);
+        fix(list, left2, left3);
+        list.add(e);
+      }
+      if (!list.equals(call.getOperands())) {
+        return call.clone(call.getType(), list);
+      }
+      return call;
+    case INPUT_REF:
+    case LITERAL:
+      return node;
+    default:
+      ImmutableBitSet bits = RelOptUtil.InputFinder.bits(node);
+      final int mid = leftCount + extraLeftExprs.size();
+      switch (Side.of(bits, mid)) {
+      case LEFT:
+        fix(extraRightExprs, mid, mid + 1);
+        extraLeftExprs.add(node);
+        return new RexInputRef(mid, node.getType());
+      case RIGHT:
+        final int index2 = mid + rightCount + extraRightExprs.size();
+        extraRightExprs.add(node);
+        return new RexInputRef(index2, node.getType());
+      case BOTH:
+      case EMPTY:
+      default:
+        return node;
+      }
+    }
+  }
+
+  private void fix(List<RexNode> operands, int before, int after) {
+    if (before == after) {
+      return;
+    }
+    for (int i = 0; i < operands.size(); i++) {
+      RexNode node = operands.get(i);
+      operands.set(i, RexUtil.shift(node, before, after - before));
+    }
+  }
+
+  /**
+   * Categorizes whether a bit set contains bits left and right of a
+   * line.
+   */
+  enum Side {
+    LEFT, RIGHT, BOTH, EMPTY;
+
+    static Side of(ImmutableBitSet bitSet, int middle) {
+      final int firstBit = bitSet.nextSetBit(0);
+      if (firstBit < 0) {
+        return EMPTY;
+      }
+      if (firstBit >= middle) {
+        return RIGHT;
+      }
+      if (bitSet.nextSetBit(middle) < 0) {
+        return LEFT;
+      }
+      return BOTH;
+    }
+  }
+
+  /**
+   * Determines whether a subquery is non-correlated. Note that a
+   * non-correlated subquery can contain correlated references, provided those
+   * references do not reference select statements that are parents of the
+   * subquery.
+   *
+   * @param subq the subquery
+   * @param bb   blackboard used while converting the subquery, i.e., the
+   *             blackboard of the parent query of this subquery
+   * @return true if the subquery is non-correlated.
+   */
+  private boolean isSubQueryNonCorrelated(RelNode subq, Blackboard bb) {
+    Set<String> correlatedVariables = RelOptUtil.getVariablesUsed(subq);
+    for (String correlName : correlatedVariables) {
+      DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
+      String originalRelName = lookup.getOriginalRelName();
+
+      int[] nsIndexes = {-1};
+      final SqlValidatorScope[] ancestorScopes = {null};
+      SqlValidatorNamespace foundNs =
+          lookup.bb.scope.resolve(
+              ImmutableList.of(originalRelName),
+              ancestorScopes,
+              nsIndexes);
+
+      assert foundNs != null;
+      assert nsIndexes.length == 1;
+
+      SqlValidatorScope ancestorScope = ancestorScopes[0];
+
+      // If the correlated reference is in a scope that's "above" the
+      // subquery, then this is a correlated subquery.
+      SqlValidatorScope parentScope = bb.scope;
+      do {
+        if (ancestorScope == parentScope) {
+          return false;
+        }
+        if (parentScope instanceof DelegatingScope) {
+          parentScope = ((DelegatingScope) parentScope).getParent();
+        } else {
+          break;
+        }
+      } while (parentScope != null);
+    }
+    return true;
+  }
+
+  /**
+   * Returns a list of fields to be prefixed to each relational expression.
+   *
+   * @return List of system fields
+   */
+  protected List<RelDataTypeField> getSystemFields() {
+    return Collections.emptyList();
+  }
+
+  private RexNode convertJoinCondition(
+      Blackboard bb,
+      SqlNode condition,
+      JoinConditionType conditionType,
+      RelNode leftRel,
+      RelNode rightRel) {
+    if (condition == null) {
+      return rexBuilder.makeLiteral(true);
+    }
+    bb.setRoot(ImmutableList.of(leftRel, rightRel));
+    replaceSubqueries(bb, condition, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+    switch (conditionType) {
+    case ON:
+      bb.setRoot(ImmutableList.of(leftRel, rightRel));
+      return bb.convertExpression(condition);
+    case USING:
+      SqlNodeList list = (SqlNodeList) condition;
+      List<String> nameList = new ArrayList<String>();
+      for (SqlNode columnName : list) {
+        final SqlIdentifier id = (SqlIdentifier) columnName;
+        String name = id.getSimple();
+        nameList.add(name);
+      }
+      return convertUsing(leftRel, rightRel, nameList);
+    default:
+      throw Util.unexpected(conditionType);
+    }
+  }
+
+  /**
+   * Returns an expression for matching columns of a USING clause or inferred
+   * from NATURAL JOIN. "a JOIN b USING (x, y)" becomes "a.x = b.x AND a.y =
+   * b.y". Returns null if the column list is empty.
+   *
+   * @param leftRel  Left input to the join
+   * @param rightRel Right input to the join
+   * @param nameList List of column names to join on
+   * @return Expression to match columns from name list, or true if name list
+   * is empty
+   */
+  private RexNode convertUsing(
+      RelNode leftRel,
+      RelNode rightRel,
+      List<String> nameList) {
+    final List<RexNode> list = Lists.newArrayList();
+    for (String name : nameList) {
+      final RelDataType leftRowType = leftRel.getRowType();
+      RelDataTypeField leftField = catalogReader.field(leftRowType, name);
+      RexNode left =
+          rexBuilder.makeInputRef(
+              leftField.getType(),
+              leftFiel

<TRUNCATED>