FAQ
Repository: hive
Updated Branches:
   refs/heads/llap 28d1082b4 -> a7b0ca733


HIVE-13295: Improvement to LDAP search queries in HS2 LDAP Authenticator (Naveen Gangam via Chaoyu Tang)


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

Branch: refs/heads/llap
Commit: e665f020b419cf9096006c45f4afcda13fa9e882
Parents: 55383d8
Author: ctang <ctang@cloudera.com>
Authored: Thu Mar 24 09:34:59 2016 -0700
Committer: ctang <ctang@cloudera.com>
Committed: Thu Mar 24 09:34:59 2016 -0700

----------------------------------------------------------------------
  .../org/apache/hadoop/hive/conf/HiveConf.java | 9 +
  .../auth/LdapAuthenticationProviderImpl.java | 317 ++++++++++---------
  .../auth/TestLdapAtnProviderWithMiniDS.java | 200 +++++++++++-
  3 files changed, 373 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/e665f020/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index b8b9dcf..b8870f2 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2237,6 +2237,15 @@ public class HiveConf extends Configuration {
      HIVE_SERVER2_PLAIN_LDAP_USERFILTER("hive.server2.authentication.ldap.userFilter", null,
          "COMMA-separated list of LDAP usernames (just short names, not full DNs).\n" +
          "For example: hiveuser,impalauser,hiveadmin,hadoopadmin"),
+ HIVE_SERVER2_PLAIN_LDAP_GUIDKEY("hive.server2.authentication.ldap.guidKey", "uid",
+ "LDAP attribute name whose values are unique in this LDAP server.\n" +
+ "For example: uid or CN."),
+ HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY("hive.server2.authentication.ldap.groupMembershipKey", "member",
+ "LDAP attribute name on the user entry that references a group, the user belongs to.\n" +
+ "For example: member, uniqueMember or memberUid"),
+ HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY("hive.server2.authentication.ldap.groupClassKey", "groupOfNames",
+ "LDAP attribute name on the group entry that is to be used in LDAP group searches.\n" +
+ "For example: group, groupOfNames or groupOfUniqueNames."),
      HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY("hive.server2.authentication.ldap.customLDAPQuery", null,
          "A full LDAP query that LDAP Atn provider uses to execute against LDAP Server.\n" +
          "If this query returns a null resultset, the LDAP Provider fails the Authentication\n" +

http://git-wip-us.apache.org/repos/asf/hive/blob/e665f020/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
index 9b0b14d..8f64672 100644
--- a/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
+++ b/service/src/java/org/apache/hive/service/auth/LdapAuthenticationProviderImpl.java
@@ -41,7 +41,6 @@ import org.slf4j.LoggerFactory;
  public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvider {

    private static final Logger LOG = LoggerFactory.getLogger(LdapAuthenticationProviderImpl.class);
- private static final String DN_ATTR = "distinguishedName";

    private String ldapURL;
    private String baseDN;
@@ -51,6 +50,9 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
    private static List<String> userFilter;
    private static List<String> groupFilter;
    private String customQuery;
+ private static String guid_attr;
+ private static String groupMembership_attr;
+ private static String groupClass_attr;

    LdapAuthenticationProviderImpl(HiveConf conf) {
      init(conf);
@@ -61,65 +63,66 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
      baseDN = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_BASEDN);
      ldapDomain = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_DOMAIN);
      customQuery = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_CUSTOMLDAPQUERY);
-
- if (customQuery == null) {
- groupBases = new ArrayList<String>();
- userBases = new ArrayList<String>();
- String groupDNPatterns = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN);
- String groupFilterVal = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER);
- String userDNPatterns = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
- String userFilterVal = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER);
-
- // parse COLON delimited root DNs for users/groups that may or may not be under BaseDN.
- // Expect the root DNs be fully qualified including the baseDN
- if (groupDNPatterns != null && groupDNPatterns.trim().length() > 0) {
- String[] groupTokens = groupDNPatterns.split(":");
- for (int i = 0; i < groupTokens.length; i++) {
- if (groupTokens[i].contains(",") && groupTokens[i].contains("=")) {
- groupBases.add(groupTokens[i]);
- } else {
- LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
- + "..ignoring " + groupTokens[i]);
- }
+ guid_attr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GUIDKEY);
+ groupBases = new ArrayList<String>();
+ userBases = new ArrayList<String>();
+ userFilter = new ArrayList<String>();
+ groupFilter = new ArrayList<String>();
+
+ String groupDNPatterns = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN);
+ String groupFilterVal = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPFILTER);
+ String userDNPatterns = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN);
+ String userFilterVal = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERFILTER);
+ groupMembership_attr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY);
+ groupClass_attr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPCLASS_KEY);
+
+ // parse COLON delimited root DNs for users/groups that may or may not be under BaseDN.
+ // Expect the root DNs be fully qualified including the baseDN
+ if (groupDNPatterns != null && groupDNPatterns.trim().length() > 0) {
+ String[] groupTokens = groupDNPatterns.split(":");
+ for (int i = 0; i < groupTokens.length; i++) {
+ if (groupTokens[i].contains(",") && groupTokens[i].contains("=")) {
+ groupBases.add(groupTokens[i]);
+ } else {
+ LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_GROUPDNPATTERN
+ + "..ignoring " + groupTokens[i]);
          }
- } else if (baseDN != null) {
- groupBases.add("uid=%s," + baseDN);
        }
+ } else if (baseDN != null) {
+ groupBases.add(guid_attr + "=%s," + baseDN);
+ }

- if (groupFilterVal != null && groupFilterVal.trim().length() > 0) {
- groupFilter = new ArrayList<String>();
- String[] groups = groupFilterVal.split(",");
- for (int i = 0; i < groups.length; i++) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Filtered group: " + groups[i]);
- }
- groupFilter.add(groups[i]);
+ if (groupFilterVal != null && groupFilterVal.trim().length() > 0) {
+ String[] groups = groupFilterVal.split(",");
+ for (int i = 0; i < groups.length; i++) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Filtered group: " + groups[i]);
          }
+ groupFilter.add(groups[i]);
        }
+ }

- if (userDNPatterns != null && userDNPatterns.trim().length() > 0) {
- String[] userTokens = userDNPatterns.split(":");
- for (int i = 0; i < userTokens.length; i++) {
- if (userTokens[i].contains(",") && userTokens[i].contains("=")) {
- userBases.add(userTokens[i]);
- } else {
- LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
- + "..ignoring " + userTokens[i]);
- }
+ if (userDNPatterns != null && userDNPatterns.trim().length() > 0) {
+ String[] userTokens = userDNPatterns.split(":");
+ for (int i = 0; i < userTokens.length; i++) {
+ if (userTokens[i].contains(",") && userTokens[i].contains("=")) {
+ userBases.add(userTokens[i]);
+ } else {
+ LOG.warn("Unexpected format for " + HiveConf.ConfVars.HIVE_SERVER2_PLAIN_LDAP_USERDNPATTERN
+ + "..ignoring " + userTokens[i]);
          }
- } else if (baseDN != null) {
- userBases.add("uid=%s," + baseDN);
        }
+ } else if (baseDN != null) {
+ userBases.add(guid_attr + "=%s," + baseDN);
+ }

- if (userFilterVal != null && userFilterVal.trim().length() > 0) {
- userFilter = new ArrayList<String>();
- String[] users = userFilterVal.split(",");
- for (int i = 0; i < users.length; i++) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Filtered user: " + users[i]);
- }
- userFilter.add(users[i]);
+ if (userFilterVal != null && userFilterVal.trim().length() > 0) {
+ String[] users = userFilterVal.split(",");
+ for (int i = 0; i < users.length; i++) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Filtered user: " + users[i]);
          }
+ userFilter.add(users[i]);
        }
      }
    }
@@ -159,7 +162,7 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
          try {
            bindDN = listIter.next().replaceAll("%s", user);
            env.put(Context.SECURITY_PRINCIPAL, bindDN);
- LOG.debug("Connecting using principal=" + user + " at url=" + ldapURL);
+ LOG.debug("Connecting using DN " + bindDN + " at url " + ldapURL);
            ctx = new InitialDirContext(env);
            break;
          } catch (NamingException e) {
@@ -168,7 +171,7 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
        }
      } else {
        env.put(Context.SECURITY_PRINCIPAL, user);
- LOG.debug("Connecting using principal=" + user + " at url=" + ldapURL);
+ LOG.debug("Connecting using principal " + user + " at url " + ldapURL);
        try {
          ctx = new InitialDirContext(env);
        } catch (NamingException e) {
@@ -177,9 +180,11 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
      }

      if (ctx == null) {
+ LOG.debug("Could not connect to the LDAP Server:Authentication failed for " + user);
        throw new AuthenticationException("LDAP Authentication failed for user", ex);
      }

+ LOG.debug("Connected using principal=" + user + " at url=" + ldapURL);
      try {
        if (isDN(user) || hasDomain(user)) {
          userName = extractName(user);
@@ -187,7 +192,24 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
          userName = user;
        }

- if (userFilter == null && groupFilter == null && customQuery == null && userBases.size() > 0) {
+ // if a custom LDAP query is specified, it takes precedence over other configuration properties.
+ // if the user being authenticated is part of the resultset from the custom query, it succeeds.
+ if (customQuery != null) {
+ List<String> resultList = executeLDAPQuery(ctx, customQuery, baseDN);
+ if (resultList != null) {
+ for (String matchedDN : resultList) {
+ LOG.info("<queried user=" + matchedDN.split(",",2)[0].split("=",2)[1] + ",user=" + user + ">");
+ if (matchedDN.split(",",2)[0].split("=",2)[1].equalsIgnoreCase(user) ||
+ matchedDN.equalsIgnoreCase(user)) {
+ LOG.info("Authentication succeeded based on result set from LDAP query");
+ return;
+ }
+ }
+ }
+ LOG.info("Authentication failed based on result set from custom LDAP query");
+ throw new AuthenticationException("Authentication failed: LDAP query " +
+ "from property returned no data");
+ } else if (userBases.size() > 0) {
          if (isDN(user)) {
            userDN = findUserDNByDN(ctx, user);
          } else {
@@ -196,7 +218,7 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
            }

            if (userDN == null) {
- userDN = findUserDNByName(ctx, baseDN, userName);
+ userDN = findUserDNByName(ctx, userName);
            }
          }

@@ -205,86 +227,60 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
          if (userDN == null) {
            throw new AuthenticationException("Authentication failed: User search failed");
          }
- return;
- }
-
- if (customQuery != null) {
- List<String> resultList = executeLDAPQuery(ctx, customQuery, baseDN);
- if (resultList != null) {
- for (String matchedDN : resultList) {
- if (matchedDN.split(",",2)[0].split("=",2)[1].equalsIgnoreCase(user)) {
- LOG.info("Authentication succeeded based on result set from LDAP query");
- return;
- }
- }
- }
- throw new AuthenticationException("Authentication failed: LDAP query " +
- "from property returned no data");
- }

- // This section checks if the user satisfies the specified user filter.
- if (userFilter != null && userFilter.size() > 0) {
- LOG.info("Authenticating user " + user + " using user filter");
+ // This section checks if the user satisfies the specified user filter.
+ if (userFilter.size() > 0) {
+ LOG.info("Authenticating user " + user + " using user filter");

- boolean success = false;
- for (String filteredUser : userFilter) {
- if (filteredUser.equalsIgnoreCase(userName)) {
- LOG.debug("User filter partially satisfied");
- success = true;
- break;
+ if (userDN != null) {
+ LOG.info("User filter partially satisfied");
            }
- }
-
- if (!success) {
- LOG.info("Authentication failed based on user membership");
- throw new AuthenticationException("Authentication failed: User not a member " +
- "of specified list");
- }
-
- userDN = findUserDNByPattern(ctx, userName);
- if (userDN != null) {
- LOG.info("User filter entirely satisfied");
- } else {
- LOG.info("User " + user + " could not be found in the configured UserBaseDN," +
- "authentication failed");
- throw new AuthenticationException("Authentication failed: UserDN could not be " +
- "found in specified User base(s)");
- }
- }

- if (groupFilter != null && groupFilter.size() > 0) {
- LOG.debug("Authenticating user " + user + " using group membership:");
+ boolean success = false;
+ for (String filteredUser : userFilter) {
+ if (filteredUser.equalsIgnoreCase(userName)) {
+ LOG.debug("User filter entirely satisfied");
+ success = true;
+ break;
+ }
+ }

- // if only groupFilter is configured.
- if (userDN == null) {
- userDN = findUserDNByName(ctx, baseDN, userName);
+ if (!success) {
+ LOG.info("Authentication failed based on user membership");
+ throw new AuthenticationException("Authentication failed: User not a member " +
+ "of specified list");
+ }
          }

- List<String> userGroups = getGroupsForUser(ctx, userDN);
- if (LOG.isDebugEnabled()) {
- LOG.debug("User member of :");
- prettyPrint(userGroups);
- }
+ // This section checks if the user satisfies the specified user filter.
+ if (groupFilter.size() > 0) {
+ LOG.debug("Authenticating user " + user + " using group membership");
+ List<String> userGroups = getGroupsForUser(ctx, userDN);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("User member of :");
+ prettyPrint(userGroups);
+ }

- if (userGroups != null) {
- for (String elem : userGroups) {
- String shortName = ((elem.split(","))[0].split("="))[1];
- String groupDN = elem.split(",", 2)[1];
- LOG.debug("Checking group:DN=" + elem + ",shortName=" + shortName +
- ",groupDN=" + groupDN);
- if (groupFilter.contains(shortName)) {
- LOG.info("Authentication succeeded based on group membership");
- return;
+ if (userGroups != null) {
+ for (String elem : userGroups) {
+ String shortName = ((elem.split(","))[0].split("="))[1];
+ if (groupFilter.contains(shortName)) {
+ LOG.info("Authentication succeeded based on group membership");
+ return;
+ }
              }
            }
- }

- throw new AuthenticationException("Authentication failed: User not a member of " +
- "listed groups");
+ LOG.debug("Authentication failed: User is not a member of configured groups");
+ throw new AuthenticationException("Authentication failed: User not a member of " +
+ "listed groups");
+ }
+ LOG.info("Authentication succeeded using ldap user search");
+ return;
        }
-
+ // Ideally we should not be here. Indicates partially configured LDAP Service.
+ // We allow it for now for backward compatibility.
        LOG.info("Simple password authentication succeeded");
-
      } catch (NamingException e) {
        throw new AuthenticationException("LDAP Authentication failed for user", e);
      } finally {
@@ -337,7 +333,7 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
     */
    public static String findGroupDNByName(DirContext ctx, String baseDN, String groupName)
      throws NamingException {
- String searchFilter = "(&(objectClass=group)(CN=" + groupName + "))";
+ String searchFilter = "(&(objectClass=" + groupClass_attr + ")(" + guid_attr + "=" + groupName + "))";
      List<String> results = null;

      results = findDNByName(ctx, baseDN, searchFilter, 2);
@@ -410,9 +406,9 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
     * @param userName A unique userid that is to be located in the LDAP.
     * @return LDAP DN if the user is found in LDAP, null otherwise.
     */
- public static String findUserDNByName(DirContext ctx, String baseDN, String userName)
+ public static String findUserDNByName(DirContext ctx, String userName)
        throws NamingException {
- if (baseDN == null) {
+ if (userBases.size() == 0) {
        return null;
      }

@@ -421,23 +417,28 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
                               "(|(uid=" + userName + ")(sAMAccountName=" + userName + ")))",
                               "(|(cn=*" + userName + "*)))"
                             };
- String searchFilter = null;
- List<String> results = null;
+
+ String searchFilter = null;
+ List<String> results = null;
+ ListIterator<String> listIter = userBases.listIterator();

      for (int i = 0; i < suffix.length; i++) {
        searchFilter = baseFilter + suffix[i];
- results = findDNByName(ctx, baseDN, searchFilter, 2);

- if(results == null) {
- continue;
- }
+ while (listIter.hasNext()) {
+ results = findDNByName(ctx, listIter.next().split(",",2)[1], searchFilter, 2);

- if(results != null && results.size() > 1) {
- //make sure there is not another item available, there should be only 1 match
- LOG.info("Matched multiple users for the user: " + userName + ",returning null");
- return null;
+ if(results == null) {
+ continue;
+ }
+
+ if(results != null && results.size() > 1) {
+ //make sure there is not another item available, there should be only 1 match
+ LOG.info("Matched multiple users for the user: " + userName + ",returning null");
+ return null;
+ }
+ return results.get(0);
        }
- return results.get(0);
      }
      return null;
    }
@@ -525,37 +526,47 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi

    /**
     * This helper method finds all the groups a given user belongs to.
- * This method relies on the "memberOf" attribute being set on the user that references
- * the group the group. The returned list ONLY includes direct groups the user belongs to.
- * Parent groups of these direct groups are NOT included.
+ * This method relies on the attribute,configurable via HIVE_SERVER2_PLAIN_LDAP_GROUPMEMBERSHIP_KEY,
+ * being set on the user entry that references the group. The returned list ONLY includes direct
+ * groups the user belongs to. Parent groups of these direct groups are NOT included.
     * @param ctx DirContext for the LDAP Connection.
- * @param userName A unique userid that is to be located in the LDAP.
+ * @param userDN A unique userDN that is to be located in the LDAP.
     * @return List of Group DNs the user belongs to, emptylist otherwise.
     */
    public static List<String> getGroupsForUser(DirContext ctx, String userDN)
        throws NamingException {
      List<String> groupList = new ArrayList<String>();
- String searchFilter = "(" + DN_ATTR + "=" + userDN + ")";
+ String user = extractName(userDN);
+ String searchFilter = "(&(objectClass=" + groupClass_attr + ")(|(" +
+ groupMembership_attr + "=" + userDN + ")(" +
+ groupMembership_attr + "=" + user + ")))";
      SearchControls searchControls = new SearchControls();
+ NamingEnumeration<SearchResult> results = null;
+ SearchResult result = null;
+ String groupBase = null;

      LOG.debug("getGroupsForUser:searchFilter=" + searchFilter);
- String[] attrIDs = { "memberOf" };
+ String[] attrIDs = new String[0];
      searchControls.setSearchScope(SearchControls.SUBTREE_SCOPE);
      searchControls.setReturningAttributes(attrIDs);

- // treat everything after the first COMMA as a baseDN for the search to find this user
- NamingEnumeration<SearchResult> results = ctx.search(userDN.split(",",2)[1], searchFilter,
- searchControls);
- while(results.hasMoreElements()) {
- NamingEnumeration<? extends Attribute> groups = results.next().getAttributes().getAll();
- while (groups.hasMore()) {
- Attribute attr = groups.next();
- NamingEnumeration<?> list = attr.getAll();
- while (list.hasMore()) {
- groupList.add((String)list.next());
+ ListIterator<String> listIter = groupBases.listIterator();
+ while (listIter.hasNext()) {
+ try {
+ groupBase = listIter.next().split(",", 2)[1];
+ LOG.debug("Searching for groups under " + groupBase);
+ results = ctx.search(groupBase, searchFilter, searchControls);
+
+ while(results.hasMoreElements()) {
+ result = results.nextElement();
+ LOG.debug("Found Group:" + result.getNameInNamespace());
+ groupList.add(result.getNameInNamespace());
          }
+ } catch (NamingException e) {
+ LOG.warn("Exception searching for user groups", e);
        }
      }
+
      return groupList;
    }

@@ -577,6 +588,10 @@ public class LdapAuthenticationProviderImpl implements PasswdAuthenticationProvi
     */
    public static List<String> executeLDAPQuery(DirContext ctx, String query, String rootDN)
        throws NamingException {
+ if (rootDN == null) {
+ return null;
+ }
+
      SearchControls searchControls = new SearchControls();
      List<String> list = new ArrayList<String>();
      String[] returnAttributes = new String[0]; //empty set

http://git-wip-us.apache.org/repos/asf/hive/blob/e665f020/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
----------------------------------------------------------------------
diff --git a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
index 832ebdf..ee9262a 100644
--- a/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
+++ b/service/src/test/org/apache/hive/service/auth/TestLdapAtnProviderWithMiniDS.java
@@ -109,21 +109,23 @@ partitions = {
        "dn: uid=group1,ou=Groups,dc=example,dc=com",
        "distinguishedName: uid=group1,ou=Groups,dc=example,dc=com",
        "objectClass: top",
- "objectClass: organizationalUnit",
+ "objectClass: groupOfNames",
        "objectClass: ExtensibleObject",
        "cn: group1",
        "ou: Groups",
        "sn: group1",
+ "member: uid=user1,ou=People,dc=example,dc=com",

        "dn: uid=group2,ou=Groups,dc=example,dc=com",
        "distinguishedName: uid=group2,ou=Groups,dc=example,dc=com",
        "objectClass: top",
- "objectClass: organizationalUnit",
+ "objectClass: groupOfNames",
        "objectClass: ExtensibleObject",
        "givenName: Group2",
        "ou: Groups",
        "cn: group1",
        "sn: group1",
+ "member: uid=user2,ou=People,dc=example,dc=com",

        "dn: uid=user1,ou=People,dc=example,dc=com",
        "distinguishedName: uid=user1,ou=People,dc=example,dc=com",
@@ -535,4 +537,198 @@ public class TestLdapAtnProviderWithMiniDS extends AbstractLdapTestUnit {
      }
    }

+ @Test
+ public void testUserFilterPositive() throws Exception {
+ String user;
+ Map<String, String> ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user2");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user2,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user2");
+ assertTrue("testUserFilterPositive: Authentication succeeded for " + user + " as expected", true);
+
+ user = "user2";
+ ldapProvider.Authenticate(user, "user2");
+ assertTrue("testUserFilterPositive: Authentication succeeded for " + user + " as expected", true);
+ } catch (AuthenticationException e) {
+ Assert.fail("testUserFilterPositive: Authentication failed for " + user + ",user expected to pass userfilter");
+ }
+
+ ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user1");
+ initLdapAtn(ldapProperties);
+
+ try {
+ user = "uid=user1,ou=People,dc=example,dc=com";
+ ldapProvider.Authenticate(user, "user1");
+ assertTrue("testUserFilterPositive: Authentication succeeded for " + user + " as expected", true);
+
+ user = "user1";
+ ldapProvider.Authenticate(user, "user1");
+ assertTrue("testUserFilterPositive: Authentication succeeded for " + user + " as expected", true);
+ } catch (AuthenticationException e) {
+ Assert.fail("testUserFilterPositive: Authentication failed for " + user + ",user expected to pass userfilter");
+ }
+
+ ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user2,user1");
+ initLdapAtn(ldapProperties);
+
+ try {
+ user = "uid=user1,ou=People,dc=example,dc=com";
+ ldapProvider.Authenticate(user, "user1");
+ assertTrue("testUserFilterPositive: Authentication succeeded for " + user + " as expected", true);
+
+ user = "user2";
+ ldapProvider.Authenticate(user, "user2");
+ assertTrue("testUserFilterPositive: Authentication succeeded for " + user + " as expected", true);
+
+ } catch (AuthenticationException e) {
+ Assert.fail("testUserFilterPositive: Authentication failed for user, user is expected to pass userfilter");
+ }
+ }
+
+ @Test
+ public void testUserFilterNegative() throws Exception {
+ String user;
+ Map<String, String> ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user2");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user1,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user1");
+ Assert.fail("testUserFilterNegative: Authentication succeeded for " + user + ",user is expected to fail userfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+
+ user = "user1";
+ try {
+ ldapProvider.Authenticate(user, "user1");
+ Assert.fail("testUserFilterNegative: Authentication succeeded for " + user + ",user is expected to fail userfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+
+ ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user1");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user2,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user2");
+ Assert.fail("testUserFilterNegative: Authentication succeeded for " + user + ",user is expected to fail userfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+
+ user = "user2";
+ try {
+ ldapProvider.Authenticate(user, "user2");
+ Assert.fail("testUserFilterNegative: Authentication succeeded for " + user + ",user is expected to fail userfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+
+ ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.userFilter", "user3");
+ initLdapAtn(ldapProperties);
+
+ user = "user1";
+ try {
+ ldapProvider.Authenticate(user, "user1");
+ Assert.fail("testUserFilterNegative: Authentication succeeded for " + user + ",user expected to fail userfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+
+ user = "uid=user2,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user2");
+ Assert.fail("testUserFilterNegative: Authentication succeeded for " + user + ",user expected to fail userfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testUserFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+ }
+
+ @Test
+ public void testGroupFilterPositive() throws Exception {
+ String user;
+ Map<String, String> ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group1,group2");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user1,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user1");
+ assertTrue("testGroupFilterPositive: Authentication succeeded for " + user + " as expected", true);
+
+ user = "user1";
+ ldapProvider.Authenticate(user, "user1");
+ assertTrue("testGroupFilterPositive: Authentication succeeded for " + user + " as expected", true);
+
+ user = "uid=user2,ou=People,dc=example,dc=com";
+ ldapProvider.Authenticate(user, "user2");
+ assertTrue("testGroupFilterPositive: Authentication succeeded for " + user + " as expected", true);
+ } catch (AuthenticationException e) {
+ Assert.fail("testGroupFilterPositive: Authentication failed for " + user + ",user expected to pass groupfilter");
+ }
+
+ ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group2");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user2,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user2");
+ assertTrue("testGroupFilterPositive: Authentication succeeded for " + user + " as expected", true);
+ } catch (AuthenticationException e) {
+ Assert.fail("testGroupFilterPositive: Authentication failed for " + user + ",user expected to pass groupfilter");
+ }
+ }
+
+ @Test
+ public void testGroupFilterNegative() throws Exception {
+ String user;
+ Map<String, String> ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group1");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user2,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user2");
+ Assert.fail("testGroupFilterNegative: Authentication succeeded for " + user + ",user expected to fail groupfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testGroupFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+
+ ldapProperties = new HashMap<String, String>();
+ ldapProperties.put("hive.server2.authentication.ldap.userDNPattern", "uid=%s,ou=People,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupDNPattern", "uid=%s,ou=Groups,dc=example,dc=com");
+ ldapProperties.put("hive.server2.authentication.ldap.groupFilter", "group2");
+ initLdapAtn(ldapProperties);
+
+ user = "uid=user1,ou=People,dc=example,dc=com";
+ try {
+ ldapProvider.Authenticate(user, "user1");
+ Assert.fail("testGroupFilterNegative: Authentication succeeded for " + user + ",user expected to fail groupfilter");
+ } catch (AuthenticationException e) {
+ assertTrue("testGroupFilterNegative: Authentication failed for " + user + " as expected", true);
+ }
+ }
  }

Search Discussions

  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12616 : NullPointerException when spark session is reused to run a mapjoin (Nemon Lou, via Szehon)


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

    Branch: refs/heads/llap
    Commit: d469e61108a1844fcc173674bfb2cd9f7ad01c18
    Parents: 219d352
    Author: Szehon Ho <szehon@cloudera.com>
    Authored: Thu Mar 24 11:12:08 2016 -0700
    Committer: Szehon Ho <szehon@cloudera.com>
    Committed: Thu Mar 24 11:12:50 2016 -0700

    ----------------------------------------------------------------------
      .../apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java | 4 ++++
      1 file changed, 4 insertions(+)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/d469e611/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
    index 1798622..2427321 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/HiveSparkClientFactory.java
    @@ -28,6 +28,7 @@ import java.util.Set;

      import org.apache.commons.compress.utils.CharsetNames;
      import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
    +import org.apache.hadoop.hive.ql.session.SessionState;
      import org.slf4j.Logger;
      import org.slf4j.LoggerFactory;
      import org.apache.hadoop.hbase.HBaseConfiguration;
    @@ -120,6 +121,9 @@ public class HiveSparkClientFactory {
            sparkMaster = sparkConf.get("spark.master");
            hiveConf.set("spark.master", sparkMaster);
          }
    + if (SessionState.get() != null && SessionState.get().getConf() != null) {
    + SessionState.get().getConf().set("spark.master", sparkMaster);
    + }
          if (sparkMaster.equals("yarn-cluster")) {
            sparkConf.put("spark.yarn.maxAppAttempts", "1");
          }
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13300 : Hive on spark throws exception for multi-insert with join (Szehon, reviewed by Xuefu and Chao Sun)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/219d3527
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/219d3527
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/219d3527

    Branch: refs/heads/llap
    Commit: 219d3527cfac09045f0ac247821746e7c95dcb8c
    Parents: e665f02
    Author: Szehon Ho <szehon@cloudera.com>
    Authored: Thu Mar 24 11:08:04 2016 -0700
    Committer: Szehon Ho <szehon@cloudera.com>
    Committed: Thu Mar 24 11:09:10 2016 -0700

    ----------------------------------------------------------------------
      .../test/resources/testconfiguration.properties | 1 +
      .../ql/exec/spark/SparkReduceRecordHandler.java | 2 +
      .../clientpositive/multi_insert_with_join.q | 29 +++++
      .../clientpositive/multi_insert_with_join.q.out | 128 +++++++++++++++++++
      .../spark/multi_insert_with_join.q.out | 128 +++++++++++++++++++
      5 files changed, 288 insertions(+)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/219d3527/itests/src/test/resources/testconfiguration.properties
    ----------------------------------------------------------------------
    diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
    index 232e262..f8e8bda 100644
    --- a/itests/src/test/resources/testconfiguration.properties
    +++ b/itests/src/test/resources/testconfiguration.properties
    @@ -1006,6 +1006,7 @@ spark.query.files=add_part_multiple.q, \
        multi_insert_lateral_view.q, \
        multi_insert_mixed.q, \
        multi_insert_move_tasks_share_dependencies.q, \
    + multi_insert_with_join.q, \
        multi_join_union.q, \
        multi_join_union_src.q, \
        multigroupby_singlemr.q, \

    http://git-wip-us.apache.org/repos/asf/hive/blob/219d3527/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
    index 439e0df..0d31e5f 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkReduceRecordHandler.java
    @@ -230,8 +230,10 @@ public class SparkReduceRecordHandler extends SparkRecordHandler {
            if (isTagged) {
              // remove the tag from key coming out of reducer
              // and store it in separate variable.
    + // make a copy for multi-insert with join case as Spark re-uses input key from same parent
              int size = keyWritable.getSize() - 1;
              tag = keyWritable.get()[size];
    + keyWritable = new BytesWritable(keyWritable.getBytes(), size);
              keyWritable.setSize(size);
            }


    http://git-wip-us.apache.org/repos/asf/hive/blob/219d3527/ql/src/test/queries/clientpositive/multi_insert_with_join.q
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/queries/clientpositive/multi_insert_with_join.q b/ql/src/test/queries/clientpositive/multi_insert_with_join.q
    new file mode 100644
    index 0000000..862dd9f
    --- /dev/null
    +++ b/ql/src/test/queries/clientpositive/multi_insert_with_join.q
    @@ -0,0 +1,29 @@
    +set hive.auto.convert.join=false;
    +
    +drop table if exists status_updates;
    +drop table if exists profiles;
    +drop table if exists school_summary;
    +drop table if exists gender_summary;
    +
    +create table status_updates(userid int,status string,ds string);
    +create table profiles(userid int,school string,gender int);
    +create table school_summary(school string,cnt int) partitioned by (ds string);
    +create table gender_summary(gender int, cnt int) partitioned by (ds string);
    +
    +insert into status_updates values (1, "status_1", "2009-03-20");
    +insert into profiles values (1, "school_1", 0);
    +
    +FROM (SELECT a.status, b.school, b.gender
    +FROM status_updates a JOIN profiles b
    +ON (a.userid = b.userid and
    +a.ds='2009-03-20' )
    +) subq1
    +INSERT OVERWRITE TABLE gender_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.gender, COUNT(1) GROUP BY subq1.gender
    +INSERT OVERWRITE TABLE school_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.school, COUNT(1) GROUP BY subq1.school;
    +
    +select * from school_summary;
    +select * from gender_summary;
    \ No newline at end of file

    http://git-wip-us.apache.org/repos/asf/hive/blob/219d3527/ql/src/test/results/clientpositive/multi_insert_with_join.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/multi_insert_with_join.q.out b/ql/src/test/results/clientpositive/multi_insert_with_join.q.out
    new file mode 100644
    index 0000000..28bce84
    --- /dev/null
    +++ b/ql/src/test/results/clientpositive/multi_insert_with_join.q.out
    @@ -0,0 +1,128 @@
    +PREHOOK: query: drop table if exists status_updates
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists status_updates
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: drop table if exists profiles
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists profiles
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: drop table if exists school_summary
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists school_summary
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: drop table if exists gender_summary
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists gender_summary
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: create table status_updates(userid int,status string,ds string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@status_updates
    +POSTHOOK: query: create table status_updates(userid int,status string,ds string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@status_updates
    +PREHOOK: query: create table profiles(userid int,school string,gender int)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@profiles
    +POSTHOOK: query: create table profiles(userid int,school string,gender int)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@profiles
    +PREHOOK: query: create table school_summary(school string,cnt int) partitioned by (ds string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@school_summary
    +POSTHOOK: query: create table school_summary(school string,cnt int) partitioned by (ds string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@school_summary
    +PREHOOK: query: create table gender_summary(gender int, cnt int) partitioned by (ds string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@gender_summary
    +POSTHOOK: query: create table gender_summary(gender int, cnt int) partitioned by (ds string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@gender_summary
    +PREHOOK: query: insert into status_updates values (1, "status_1", "2009-03-20")
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@values__tmp__table__1
    +PREHOOK: Output: default@status_updates
    +POSTHOOK: query: insert into status_updates values (1, "status_1", "2009-03-20")
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@values__tmp__table__1
    +POSTHOOK: Output: default@status_updates
    +POSTHOOK: Lineage: status_updates.ds SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
    +POSTHOOK: Lineage: status_updates.status SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
    +POSTHOOK: Lineage: status_updates.userid EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
    +PREHOOK: query: insert into profiles values (1, "school_1", 0)
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@values__tmp__table__2
    +PREHOOK: Output: default@profiles
    +POSTHOOK: query: insert into profiles values (1, "school_1", 0)
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@values__tmp__table__2
    +POSTHOOK: Output: default@profiles
    +POSTHOOK: Lineage: profiles.gender EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
    +POSTHOOK: Lineage: profiles.school SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
    +POSTHOOK: Lineage: profiles.userid EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
    +PREHOOK: query: FROM (SELECT a.status, b.school, b.gender
    +FROM status_updates a JOIN profiles b
    +ON (a.userid = b.userid and
    +a.ds='2009-03-20' )
    +) subq1
    +INSERT OVERWRITE TABLE gender_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.gender, COUNT(1) GROUP BY subq1.gender
    +INSERT OVERWRITE TABLE school_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.school, COUNT(1) GROUP BY subq1.school
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@profiles
    +PREHOOK: Input: default@status_updates
    +PREHOOK: Output: default@gender_summary@ds=2009-03-20
    +PREHOOK: Output: default@school_summary@ds=2009-03-20
    +POSTHOOK: query: FROM (SELECT a.status, b.school, b.gender
    +FROM status_updates a JOIN profiles b
    +ON (a.userid = b.userid and
    +a.ds='2009-03-20' )
    +) subq1
    +INSERT OVERWRITE TABLE gender_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.gender, COUNT(1) GROUP BY subq1.gender
    +INSERT OVERWRITE TABLE school_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.school, COUNT(1) GROUP BY subq1.school
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@profiles
    +POSTHOOK: Input: default@status_updates
    +POSTHOOK: Output: default@gender_summary@ds=2009-03-20
    +POSTHOOK: Output: default@school_summary@ds=2009-03-20
    +POSTHOOK: Lineage: gender_summary PARTITION(ds=2009-03-20).cnt EXPRESSION [(status_updates)a.null, (profiles)b.null, ]
    +POSTHOOK: Lineage: gender_summary PARTITION(ds=2009-03-20).gender SIMPLE [(profiles)b.FieldSchema(name:gender, type:int, comment:null), ]
    +POSTHOOK: Lineage: school_summary PARTITION(ds=2009-03-20).cnt EXPRESSION [(status_updates)a.null, (profiles)b.null, ]
    +POSTHOOK: Lineage: school_summary PARTITION(ds=2009-03-20).school SIMPLE [(profiles)b.FieldSchema(name:school, type:string, comment:null), ]
    +PREHOOK: query: select * from school_summary
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@school_summary
    +PREHOOK: Input: default@school_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +POSTHOOK: query: select * from school_summary
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@school_summary
    +POSTHOOK: Input: default@school_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +school_1 1 2009-03-20
    +PREHOOK: query: select * from gender_summary
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@gender_summary
    +PREHOOK: Input: default@gender_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +POSTHOOK: query: select * from gender_summary
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@gender_summary
    +POSTHOOK: Input: default@gender_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +0 1 2009-03-20

    http://git-wip-us.apache.org/repos/asf/hive/blob/219d3527/ql/src/test/results/clientpositive/spark/multi_insert_with_join.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/spark/multi_insert_with_join.q.out b/ql/src/test/results/clientpositive/spark/multi_insert_with_join.q.out
    new file mode 100644
    index 0000000..28bce84
    --- /dev/null
    +++ b/ql/src/test/results/clientpositive/spark/multi_insert_with_join.q.out
    @@ -0,0 +1,128 @@
    +PREHOOK: query: drop table if exists status_updates
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists status_updates
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: drop table if exists profiles
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists profiles
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: drop table if exists school_summary
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists school_summary
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: drop table if exists gender_summary
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: drop table if exists gender_summary
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: create table status_updates(userid int,status string,ds string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@status_updates
    +POSTHOOK: query: create table status_updates(userid int,status string,ds string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@status_updates
    +PREHOOK: query: create table profiles(userid int,school string,gender int)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@profiles
    +POSTHOOK: query: create table profiles(userid int,school string,gender int)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@profiles
    +PREHOOK: query: create table school_summary(school string,cnt int) partitioned by (ds string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@school_summary
    +POSTHOOK: query: create table school_summary(school string,cnt int) partitioned by (ds string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@school_summary
    +PREHOOK: query: create table gender_summary(gender int, cnt int) partitioned by (ds string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@gender_summary
    +POSTHOOK: query: create table gender_summary(gender int, cnt int) partitioned by (ds string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@gender_summary
    +PREHOOK: query: insert into status_updates values (1, "status_1", "2009-03-20")
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@values__tmp__table__1
    +PREHOOK: Output: default@status_updates
    +POSTHOOK: query: insert into status_updates values (1, "status_1", "2009-03-20")
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@values__tmp__table__1
    +POSTHOOK: Output: default@status_updates
    +POSTHOOK: Lineage: status_updates.ds SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
    +POSTHOOK: Lineage: status_updates.status SIMPLE [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
    +POSTHOOK: Lineage: status_updates.userid EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
    +PREHOOK: query: insert into profiles values (1, "school_1", 0)
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@values__tmp__table__2
    +PREHOOK: Output: default@profiles
    +POSTHOOK: query: insert into profiles values (1, "school_1", 0)
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@values__tmp__table__2
    +POSTHOOK: Output: default@profiles
    +POSTHOOK: Lineage: profiles.gender EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col3, type:string, comment:), ]
    +POSTHOOK: Lineage: profiles.school SIMPLE [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col2, type:string, comment:), ]
    +POSTHOOK: Lineage: profiles.userid EXPRESSION [(values__tmp__table__2)values__tmp__table__2.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
    +PREHOOK: query: FROM (SELECT a.status, b.school, b.gender
    +FROM status_updates a JOIN profiles b
    +ON (a.userid = b.userid and
    +a.ds='2009-03-20' )
    +) subq1
    +INSERT OVERWRITE TABLE gender_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.gender, COUNT(1) GROUP BY subq1.gender
    +INSERT OVERWRITE TABLE school_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.school, COUNT(1) GROUP BY subq1.school
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@profiles
    +PREHOOK: Input: default@status_updates
    +PREHOOK: Output: default@gender_summary@ds=2009-03-20
    +PREHOOK: Output: default@school_summary@ds=2009-03-20
    +POSTHOOK: query: FROM (SELECT a.status, b.school, b.gender
    +FROM status_updates a JOIN profiles b
    +ON (a.userid = b.userid and
    +a.ds='2009-03-20' )
    +) subq1
    +INSERT OVERWRITE TABLE gender_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.gender, COUNT(1) GROUP BY subq1.gender
    +INSERT OVERWRITE TABLE school_summary
    +PARTITION(ds='2009-03-20')
    +SELECT subq1.school, COUNT(1) GROUP BY subq1.school
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@profiles
    +POSTHOOK: Input: default@status_updates
    +POSTHOOK: Output: default@gender_summary@ds=2009-03-20
    +POSTHOOK: Output: default@school_summary@ds=2009-03-20
    +POSTHOOK: Lineage: gender_summary PARTITION(ds=2009-03-20).cnt EXPRESSION [(status_updates)a.null, (profiles)b.null, ]
    +POSTHOOK: Lineage: gender_summary PARTITION(ds=2009-03-20).gender SIMPLE [(profiles)b.FieldSchema(name:gender, type:int, comment:null), ]
    +POSTHOOK: Lineage: school_summary PARTITION(ds=2009-03-20).cnt EXPRESSION [(status_updates)a.null, (profiles)b.null, ]
    +POSTHOOK: Lineage: school_summary PARTITION(ds=2009-03-20).school SIMPLE [(profiles)b.FieldSchema(name:school, type:string, comment:null), ]
    +PREHOOK: query: select * from school_summary
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@school_summary
    +PREHOOK: Input: default@school_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +POSTHOOK: query: select * from school_summary
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@school_summary
    +POSTHOOK: Input: default@school_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +school_1 1 2009-03-20
    +PREHOOK: query: select * from gender_summary
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@gender_summary
    +PREHOOK: Input: default@gender_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +POSTHOOK: query: select * from gender_summary
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@gender_summary
    +POSTHOOK: Input: default@gender_summary@ds=2009-03-20
    +#### A masked pattern was here ####
    +0 1 2009-03-20
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13008 - WebHcat DDL commands in secure mode NPE when default FileSystem doesn't support delegation tokens (Eugene Koifman, reviewed by Chris Nauroth, Thejas Nair)


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

    Branch: refs/heads/llap
    Commit: ab095f0bc24447ab73843a1ae23a32f7b6c4bd1a
    Parents: f9d1b6a
    Author: Eugene Koifman <ekoifman@hortonworks.com>
    Authored: Thu Mar 24 18:03:32 2016 -0700
    Committer: Eugene Koifman <ekoifman@hortonworks.com>
    Committed: Thu Mar 24 18:03:32 2016 -0700

    ----------------------------------------------------------------------
      .../hcatalog/templeton/SecureProxySupport.java | 46 ++++++++++++++------
      1 file changed, 33 insertions(+), 13 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/ab095f0b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SecureProxySupport.java
    ----------------------------------------------------------------------
    diff --git a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SecureProxySupport.java b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SecureProxySupport.java
    index 2ac62c0..13f3c9b 100644
    --- a/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SecureProxySupport.java
    +++ b/hcatalog/webhcat/svr/src/main/java/org/apache/hive/hcatalog/templeton/SecureProxySupport.java
    @@ -20,10 +20,14 @@ package org.apache.hive.hcatalog.templeton;

      import java.io.File;
      import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
      import java.security.PrivilegedExceptionAction;
    +import java.util.Collection;
      import java.util.List;
      import java.util.Map;

    +import org.apache.commons.lang3.ArrayUtils;
      import org.slf4j.Logger;
      import org.slf4j.LoggerFactory;
      import org.apache.hadoop.conf.Configuration;
    @@ -79,7 +83,7 @@ public class SecureProxySupport {
            this.user = user;
            File t = File.createTempFile("templeton", null);
            tokenPath = new Path(t.toURI());
    - Token fsToken = getFSDelegationToken(user, conf);
    + Token[] fsToken = getFSDelegationToken(user, conf);
            String hcatTokenStr;
            try {
              hcatTokenStr = buildHcatDelegationToken(user);
    @@ -130,11 +134,11 @@ public class SecureProxySupport {
          }
        }

    - class TokenWrapper {
    - Token<?> token;
    + private static class TokenWrapper {
    + Token<?>[] tokens = new Token<?>[0];
        }

    - private Token<?> getFSDelegationToken(String user,
    + private Token<?>[] getFSDelegationToken(String user,
                            final Configuration conf)
          throws IOException, InterruptedException {
          LOG.info("user: " + user + " loginUser: " + UserGroupInformation.getLoginUser().getUserName());
    @@ -142,18 +146,32 @@ public class SecureProxySupport {

          final TokenWrapper twrapper = new TokenWrapper();
          ugi.doAs(new PrivilegedExceptionAction<Object>() {
    - public Object run() throws IOException {
    - FileSystem fs = FileSystem.get(conf);
    - //todo: according to JavaDoc this seems like private API: addDelegationToken should be used
    - twrapper.token = fs.getDelegationToken(ugi.getShortUserName());
    + public Object run() throws IOException, URISyntaxException {
    + Credentials creds = new Credentials();
    + //get Tokens for default FS. Not all FSs support delegation tokens, e.g. WASB
    + collectTokens(FileSystem.get(conf), twrapper, creds, ugi.getShortUserName());
    + //get tokens for all other known FSs since Hive tables may result in different ones
    + //passing "creds" prevents duplicate tokens from being added
    + Collection<String> URIs = conf.getStringCollection("mapreduce.job.hdfs-servers");
    + for(String uri : URIs) {
    + LOG.debug("Getting tokens for " + uri);
    + collectTokens(FileSystem.get(new URI(uri), conf), twrapper, creds, ugi.getShortUserName());
    + }
              return null;
            }
          });
    - return twrapper.token;
    -
    + return twrapper.tokens;
        }
    -
    - private void writeProxyDelegationTokens(final Token<?> fsToken,
    + private static void collectTokens(FileSystem fs, TokenWrapper twrapper, Credentials creds, String userName) throws IOException {
    + Token[] tokens = fs.addDelegationTokens(userName, creds);
    + if(tokens != null && tokens.length > 0) {
    + twrapper.tokens = ArrayUtils.addAll(twrapper.tokens, tokens);
    + }
    + }
    + /**
    + * @param fsTokens not null
    + */
    + private void writeProxyDelegationTokens(final Token<?> fsTokens[],
                            final Token<?> msToken,
                            final Configuration conf,
                            String user,
    @@ -168,7 +186,9 @@ public class SecureProxySupport {
          ugi.doAs(new PrivilegedExceptionAction<Object>() {
            public Object run() throws IOException {
              Credentials cred = new Credentials();
    - cred.addToken(fsToken.getService(), fsToken);
    + for(Token<?> fsToken : fsTokens) {
    + cred.addToken(fsToken.getService(), fsToken);
    + }
              cred.addToken(msToken.getService(), msToken);
              cred.writeTokenStorageFile(tokenPath, conf);
              return null;
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12552 : Wrong number of reducer estimation causing job to fail (Rajesh Balamohan via Gunther Hagleitner)


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

    Branch: refs/heads/llap
    Commit: b75d9ea8a73f85d1420f8e3ba1e3b8f9b9acdc5e
    Parents: b1c4502
    Author: Rajesh Balamohan <rba...@...apache dot org>
    Authored: Wed Dec 9 11:48:00 2015 -0800
    Committer: Ashutosh Chauhan <hashutosh@apache.org>
    Committed: Fri Mar 25 07:21:55 2016 -0700

    ----------------------------------------------------------------------
      ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java | 3 ++-
      1 file changed, 2 insertions(+), 1 deletion(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/b75d9ea8/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
    index 8eab3af..d5a2eca 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/GenTezUtils.java
    @@ -119,7 +119,8 @@ public class GenTezUtils {

            // max we allow tez to pick
            int maxPartition = (int) (reduceSink.getConf().getNumReducers() * maxPartitionFactor);
    - maxPartition = (maxPartition > maxReducers) ? maxReducers : maxPartition;
    + maxPartition = Math.max(1, (maxPartition > maxReducers) ? maxReducers :
    + maxPartition);

            reduceWork.setMinReduceTasks(minPartition);
            reduceWork.setMaxReduceTasks(maxPartition);
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13362: Commit binary file required for HIVE-13361 (Prasanth Jayachandran reviewed by Gopal V)


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

    Branch: refs/heads/llap
    Commit: dfba1fb280f82822c1c006a0961a3ce9a52b6a6d
    Parents: ab095f0
    Author: Prasanth Jayachandran <prasanthj@apache.org>
    Authored: Thu Mar 24 20:09:14 2016 -0500
    Committer: Prasanth Jayachandran <prasanthj@apache.org>
    Committed: Thu Mar 24 20:10:26 2016 -0500

    ----------------------------------------------------------------------
      data/files/alltypesorc3xcols | Bin 0 -> 1504592 bytes
      1 file changed, 0 insertions(+), 0 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/dfba1fb2/data/files/alltypesorc3xcols
    ----------------------------------------------------------------------
    diff --git a/data/files/alltypesorc3xcols b/data/files/alltypesorc3xcols
    new file mode 100644
    index 0000000..e484873
    Binary files /dev/null and b/data/files/alltypesorc3xcols differ
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13325: Excessive logging when ORC PPD fails type conversions (Prasanth Jayachandran reviewed by Gopal V)


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

    Branch: refs/heads/llap
    Commit: d3a5f20b4487e241b3e9424d1d762dfca0c25d2f
    Parents: d469e61
    Author: Prasanth Jayachandran <prasanthj@apache.org>
    Authored: Thu Mar 24 13:30:55 2016 -0500
    Committer: Prasanth Jayachandran <prasanthj@apache.org>
    Committed: Thu Mar 24 13:31:08 2016 -0500

    ----------------------------------------------------------------------
      .../hadoop/hive/ql/io/orc/RecordReaderImpl.java | 15 ++++++++++++---
      1 file changed, 12 insertions(+), 3 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/d3a5f20b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
    index d511df6..aa835ae 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java
    @@ -378,9 +378,9 @@ public class RecordReaderImpl implements RecordReader {
          }

          TruthValue result;
    + Object baseObj = predicate.getLiteral();
          try {
            // Predicate object and stats objects are converted to the type of the predicate object.
    - Object baseObj = predicate.getLiteral();
            Object minValue = getBaseObjectForComparison(predicate.getType(), min);
            Object maxValue = getBaseObjectForComparison(predicate.getType(), max);
            Object predObj = getBaseObjectForComparison(predicate.getType(), baseObj);
    @@ -392,8 +392,17 @@ public class RecordReaderImpl implements RecordReader {
            // in case failed conversion, return the default YES_NO_NULL truth value
          } catch (Exception e) {
            if (LOG.isWarnEnabled()) {
    - LOG.warn("Exception when evaluating predicate. Skipping ORC PPD." +
    - " Exception: " + ExceptionUtils.getStackTrace(e));
    + final String statsType = min == null ?
    + (max == null ? "null" : max.getClass().getSimpleName()) :
    + min.getClass().getSimpleName();
    + final String predicateType = baseObj == null ? "null" : baseObj.getClass().getSimpleName();
    + final String reason = e.getClass().getSimpleName() + " when evaluating predicate." +
    + " Skipping ORC PPD." +
    + " Exception: " + e.getMessage() +
    + " StatsType: " + statsType +
    + " PredicateType: " + predicateType;
    + LOG.warn(reason);
    + LOG.debug(reason, e);
            }
            if (predicate.getOperator().equals(PredicateLeaf.Operator.NULL_SAFE_EQUALS) || !hasNull) {
              result = TruthValue.YES_NO;
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
    index 038e382..1e41fce 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
    @@ -23,13 +23,19 @@ import static org.junit.Assert.assertFalse;
      import static org.junit.Assert.assertTrue;

      import java.io.UnsupportedEncodingException;
    +import java.math.BigDecimal;
    +import java.math.MathContext;
    +import java.math.RoundingMode;
    +import java.sql.Timestamp;
      import java.util.Arrays;
    +import java.util.Random;
    +import java.util.concurrent.TimeUnit;

      import junit.framework.Assert;

      import org.apache.hadoop.hive.common.type.Decimal128;
      import org.apache.hadoop.hive.common.type.HiveDecimal;
    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
    +import org.apache.hadoop.hive.common.type.RandomTypeUtil;
      import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
    @@ -39,6 +45,7 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
      import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.*;
      import org.apache.hadoop.hive.ql.exec.vector.expressions.*;
      import org.apache.hadoop.hive.serde2.io.TimestampWritable;
    +import org.apache.hadoop.hive.serde2.typeinfo.HiveDecimalUtils;
      import org.junit.Test;

      /**
    @@ -84,8 +91,8 @@ public class TestVectorTypeCasts {
          b.cols[0].noNulls = true;
          VectorExpression expr = new CastDoubleToTimestamp(0, 1);
          expr.evaluate(b);
    - Assert.assertEquals(0.0, resultV.getTimestampSecondsWithFractionalNanos(3));
    - Assert.assertEquals(0.5d, resultV.getTimestampSecondsWithFractionalNanos(4));
    + Assert.assertEquals(0.0, TimestampWritable.getDouble(resultV.asScratchTimestamp(3)));
    + Assert.assertEquals(0.5d, TimestampWritable.getDouble(resultV.asScratchTimestamp(4)));
        }

        @Test
    @@ -103,39 +110,51 @@ public class TestVectorTypeCasts {

        @Test
        public void testCastLongToTimestamp() {
    - VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut();
    + long[] longValues = new long[500];
    + VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchLongInTimestampOut(longValues);
          TimestampColumnVector resultV = (TimestampColumnVector) b.cols[1];
          b.cols[0].noNulls = true;
          VectorExpression expr = new CastLongToTimestamp(0, 1);
          expr.evaluate(b);
    - Assert.assertEquals(-2, resultV.getTimestampSeconds(0));
    - Assert.assertEquals(2, resultV.getTimestampSeconds(1));
    + for (int i = 0; i < longValues.length; i++) {
    + Timestamp timestamp = resultV.asScratchTimestamp(i);
    + long actual = TimestampWritable.getLong(timestamp);
    + assertEquals(actual, longValues[i]);
    + }
        }

        @Test
        public void testCastTimestampToLong() {
    - VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut();
    + long[] longValues = new long[500];
    + VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInLongOut(longValues);
          TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
    - inV.set(0, new PisaTimestamp(0, PisaTimestamp.NANOSECONDS_PER_SECOND)); // Make one entry produce interesting result
    - // (1 sec after epoch).
    -
          LongColumnVector resultV = (LongColumnVector) b.cols[1];
          b.cols[0].noNulls = true;
          VectorExpression expr = new CastTimestampToLong(0, 1);
          expr.evaluate(b);
    - Assert.assertEquals(1, resultV.vector[0]);
    + for (int i = 0; i < longValues.length; i++) {
    + long actual = resultV.vector[i];
    + long timestampLong = inV.getTimestampAsLong(i);
    + if (actual != timestampLong) {
    + assertTrue(false);
    + }
    + }
        }

        @Test
        public void testCastTimestampToDouble() {
    - VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInDoubleOut();
    + double[] doubleValues = new double[500];
    + VectorizedRowBatch b = TestVectorMathFunctions.getVectorizedRowBatchTimestampInDoubleOut(doubleValues);
          TimestampColumnVector inV = (TimestampColumnVector) b.cols[0];
          DoubleColumnVector resultV = (DoubleColumnVector) b.cols[1];
          b.cols[0].noNulls = true;
          VectorExpression expr = new CastTimestampToDouble(0, 1);
          expr.evaluate(b);
    - Assert.assertEquals(-1E-9D , resultV.vector[1]);
    - Assert.assertEquals(1E-9D, resultV.vector[3]);
    + for (int i = 0; i < doubleValues.length; i++) {
    + double actual = resultV.vector[i];
    + double doubleValue = TimestampWritable.getDouble(inV.asScratchTimestamp(i));
    + assertEquals(actual, doubleValue, 0.000000001F);
    + }
        }

        public byte[] toBytes(String s) {
    @@ -356,16 +375,19 @@ public class TestVectorTypeCasts {

        @Test
        public void testCastDecimalToTimestamp() {
    - VectorizedRowBatch b = getBatchDecimalTimestamp();
    + double[] doubleValues = new double[500];
    + VectorizedRowBatch b = getBatchDecimalTimestamp(doubleValues);
          VectorExpression expr = new CastDecimalToTimestamp(0, 1);
          expr.evaluate(b);
          TimestampColumnVector r = (TimestampColumnVector) b.cols[1];
    - assertEquals(1111111111L, r.getNanoOfDay(0));
    - assertEquals(0L, r.getEpochDay(0));
    - assertEquals(-2222222222L, r.getNanoOfDay(1));
    - assertEquals(0L, r.getEpochDay(1));
    - assertEquals(999999999L, r.getNanoOfDay(2));
    - assertEquals(365L, r.getEpochDay(2));
    + for (int i = 0; i < doubleValues.length; i++) {
    + Timestamp timestamp = r.asScratchTimestamp(i);
    + double asDouble = TimestampWritable.getDouble(timestamp);
    + double expectedDouble = doubleValues[i];
    + if (expectedDouble != asDouble) {
    + assertTrue(false);
    + }
    + }
        }

        private VectorizedRowBatch getBatchDecimalLong2() {
    @@ -384,19 +406,25 @@ public class TestVectorTypeCasts {
          return b;
        }

    - private VectorizedRowBatch getBatchDecimalTimestamp() {
    + private VectorizedRowBatch getBatchDecimalTimestamp(double[] doubleValues) {
          VectorizedRowBatch b = new VectorizedRowBatch(2);
          DecimalColumnVector dv;
    - short scale = 9;
    - b.cols[0] = dv = new DecimalColumnVector(18, scale);
    - b.cols[1] = new TimestampColumnVector();
    -
    - b.size = 3;
    -
    - dv.vector[0].set(HiveDecimal.create("1.111111111").setScale(scale));
    - dv.vector[1].set(HiveDecimal.create("-2.222222222").setScale(scale));
    - dv.vector[2].set(HiveDecimal.create("31536000.999999999").setScale(scale));
    -
    + b.cols[0] = dv = new DecimalColumnVector(doubleValues.length, HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE);
    + b.cols[1] = new TimestampColumnVector(doubleValues.length);
    + dv.noNulls = true;
    + Random r = new Random(94830);
    + for (int i = 0; i < doubleValues.length; i++) {
    + long millis = RandomTypeUtil.randomMillis(r);
    + Timestamp ts = new Timestamp(millis);
    + int nanos = RandomTypeUtil.randomNanos(r);
    + ts.setNanos(nanos);
    + TimestampWritable tsw = new TimestampWritable(ts);
    + double asDouble = tsw.getDouble();
    + doubleValues[i] = asDouble;
    + HiveDecimal hiveDecimal = HiveDecimal.create(new BigDecimal(asDouble));
    + dv.set(i, hiveDecimal);
    + }
    + b.size = doubleValues.length;
          return b;
        }

    @@ -422,14 +450,44 @@ public class TestVectorTypeCasts {
          return b;
        }

    - private VectorizedRowBatch getBatchTimestampDecimal() {
    +
    + public static final long NANOSECONDS_PER_SECOND = TimeUnit.SECONDS.toNanos(1);
    + public static final long MILLISECONDS_PER_SECOND = TimeUnit.SECONDS.toMillis(1);
    + public static final long NANOSECONDS_PER_MILLISSECOND = TimeUnit.MILLISECONDS.toNanos(1);
    +
    + private VectorizedRowBatch getBatchTimestampDecimal(HiveDecimal[] hiveDecimalValues) {
    + Random r = new Random(994);
          VectorizedRowBatch b = new VectorizedRowBatch(2);
          TimestampColumnVector tcv;
    - b.cols[0] = tcv = new TimestampColumnVector();
    - b.cols[1] = new DecimalColumnVector(18, 2);
    - tcv.set(0, new PisaTimestamp( 0, 0));
    - tcv.set(1, new PisaTimestamp( 0, -1));
    - tcv.set(2, new PisaTimestamp( 99999999999999L / PisaTimestamp.NANOSECONDS_PER_DAY, 99999999999999L % PisaTimestamp.NANOSECONDS_PER_DAY));
    + b.cols[0] = tcv = new TimestampColumnVector(hiveDecimalValues.length);
    + b.cols[1] = new DecimalColumnVector(hiveDecimalValues.length, HiveDecimal.SYSTEM_DEFAULT_PRECISION, HiveDecimal.SYSTEM_DEFAULT_SCALE);
    + for (int i = 0; i < hiveDecimalValues.length; i++) {
    + int optionalNanos = 0;
    + switch (r.nextInt(4)) {
    + case 0:
    + // No nanos.
    + break;
    + case 1:
    + optionalNanos = r.nextInt((int) NANOSECONDS_PER_SECOND);
    + break;
    + case 2:
    + // Limit to milliseconds only...
    + optionalNanos = r.nextInt((int) MILLISECONDS_PER_SECOND) * (int) NANOSECONDS_PER_MILLISSECOND;
    + break;
    + case 3:
    + // Limit to below milliseconds only...
    + optionalNanos = r.nextInt((int) NANOSECONDS_PER_MILLISSECOND);
    + break;
    + }
    + long millis = RandomTypeUtil.randomMillis(r);
    + Timestamp ts = new Timestamp(millis);
    + ts.setNanos(optionalNanos);
    + TimestampWritable tsw = new TimestampWritable(ts);
    + hiveDecimalValues[i] = tsw.getHiveDecimal();
    +
    + tcv.set(i, ts);
    + }
    + b.size = hiveDecimalValues.length;
          return b;
        }

    @@ -440,9 +498,18 @@ public class TestVectorTypeCasts {
          expr.evaluate(b);
          DecimalColumnVector r = (DecimalColumnVector) b.cols[1];

    - assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("0.0")));
    - assertTrue(r.vector[1].getHiveDecimal().equals(HiveDecimal.create("-1.0")));
    - assertTrue(r.vector[2].getHiveDecimal().equals(HiveDecimal.create("99999999999999")));
    + HiveDecimal hd0 = HiveDecimal.create("0.0");
    + if (!hd0.equals(r.vector[0].getHiveDecimal())) {
    + assertTrue(false);
    + }
    + HiveDecimal hd1 = HiveDecimal.create("-1.0");
    + if (!hd1.equals(r.vector[1].getHiveDecimal())) {
    + assertTrue(false);
    + }
    + HiveDecimal hd2 = HiveDecimal.create("99999999999999");
    + if (!hd2.equals(r.vector[2].getHiveDecimal())) {
    + assertTrue(false);
    + }
        }

        private VectorizedRowBatch getBatchDoubleDecimal() {
    @@ -496,25 +563,37 @@ public class TestVectorTypeCasts {

          // The input timestamps are stored as long values
          // measured in nanoseconds from the epoch.
    - VectorizedRowBatch b = getBatchTimestampDecimal();
    + HiveDecimal[] hiveDecimalValues = new HiveDecimal[500];
    + VectorizedRowBatch b = getBatchTimestampDecimal(hiveDecimalValues);
          VectorExpression expr = new CastTimestampToDecimal(0, 1);
          TimestampColumnVector inT = (TimestampColumnVector) b.cols[0];
    - inT.set(1, new PisaTimestamp(0, -1990000000L));
          expr.evaluate(b);
          DecimalColumnVector r = (DecimalColumnVector) b.cols[1];
    - assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("0.00")));
    - assertTrue(r.vector[1].getHiveDecimal().equals(HiveDecimal.create("-1.99")));
    - assertTrue(r.vector[2].getHiveDecimal().equals(HiveDecimal.create("100000.00")));
    + for (int i = 0; i < hiveDecimalValues.length; i++) {
    + HiveDecimal hiveDecimal = r.vector[i].getHiveDecimal();
    + HiveDecimal expectedHiveDecimal = hiveDecimalValues[i];
    + if (!hiveDecimal.equals(expectedHiveDecimal)) {
    + assertTrue(false);
    + }
    + }

          // Try again with a value that won't fit in 5 digits, to make
          // sure that NULL is produced.
    - b = getBatchTimestampDecimalPrec5Scale2();
    + b.cols[1] = r = new DecimalColumnVector(hiveDecimalValues.length, 5, 2);
          expr.evaluate(b);
          r = (DecimalColumnVector) b.cols[1];
    - assertFalse(r.noNulls);
    - assertFalse(r.isNull[0]);
    - assertFalse(r.isNull[1]);
    - assertTrue(r.isNull[2]);
    + for (int i = 0; i < hiveDecimalValues.length; i++) {
    + HiveDecimal hiveDecimal = r.vector[i].getHiveDecimal();
    + HiveDecimal expectedHiveDecimal = hiveDecimalValues[i];
    + if (HiveDecimal.enforcePrecisionScale(expectedHiveDecimal, 5, 2) == null) {
    + assertTrue(r.isNull[i]);
    + } else {
    + assertTrue(!r.isNull[i]);
    + if (!hiveDecimal.equals(expectedHiveDecimal)) {
    + assertTrue(false);
    + }
    + }
    + }
        }

        /* This batch has output decimal column precision 5 and scale 2.
    @@ -533,41 +612,6 @@ public class TestVectorTypeCasts {
          return b;
        }

    - private VectorizedRowBatch getBatchTimestampDecimalPrec5Scale2() {
    - VectorizedRowBatch b = new VectorizedRowBatch(2);
    - TimestampColumnVector tcv;
    - b.cols[0] = tcv = new TimestampColumnVector();
    - b.cols[1] = new DecimalColumnVector(5, 2);
    - tcv.set(0, new PisaTimestamp(0, 0));
    - tcv.set(1, new PisaTimestamp(0, -1));
    - tcv.set(2, new PisaTimestamp(99999999999999L / PisaTimestamp.NANOSECONDS_PER_DAY, 99999999999999L % PisaTimestamp.NANOSECONDS_PER_DAY));
    - return b;
    - }
    -
    - /*
    - @Test
    - public void testCastDecimalToDecimal() {
    -
    - // test casting from one precision and scale to another.
    - VectorizedRowBatch b = getBatchDecimalDecimal();
    - VectorExpression expr = new CastDecimalToDecimal(0, 1);
    - expr.evaluate(b);
    - DecimalColumnVector r = (DecimalColumnVector) b.cols[1];
    - assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("10.00", (short) 2)));
    - assertFalse(r.noNulls);
    - assertTrue(r.isNull[1]);
    -
    - // test an increase in precision/scale
    - b = getBatchDecimalDecimal();
    - expr = new CastDecimalToDecimal(1, 0);
    - expr.evaluate(b);
    - r = (DecimalColumnVector) b.cols[0];
    - assertTrue(r.vector[0].getHiveDecimal().equals(HiveDecimal.create("100.01", (short) 4)));
    - assertTrue(r.vector[1].getHiveDecimal().equals(HiveDecimal.create("-200.02", (short) 4)));
    - assertTrue(r.noNulls);
    - }
    - */
    -
        private VectorizedRowBatch getBatchDecimalDecimal() {
          VectorizedRowBatch b = new VectorizedRowBatch(2);


    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
    index ab86082..98849c3 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/FakeVectorRowBatchFromObjectIterables.java
    @@ -27,7 +27,6 @@ import java.util.regex.Matcher;
      import java.util.regex.Pattern;

      import org.apache.hadoop.hive.common.type.HiveDecimal;
    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
      import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
    @@ -111,7 +110,7 @@ public class FakeVectorRowBatchFromObjectIterables extends FakeVectorRowBatchBas
                    Object value) {
                  TimestampColumnVector lcv = (TimestampColumnVector) columnVector;
                  Timestamp t = (Timestamp) value;
    - lcv.set(row, new PisaTimestamp(t));
    + lcv.set(row, t);
                }
              };


    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
    index 649e52b..84717b1 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
    @@ -22,7 +22,6 @@ import java.sql.Timestamp;
      import java.util.Random;

      import org.apache.hadoop.hive.common.type.HiveDecimal;
    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
      import org.apache.hadoop.hive.common.type.RandomTypeUtil;
      import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
    @@ -81,7 +80,7 @@ public class VectorizedRowGroupGenUtil {
        }

        public static TimestampColumnVector generateTimestampColumnVector(
    - boolean nulls, boolean repeating, int size, Random rand) {
    + boolean nulls, boolean repeating, int size, Random rand, Timestamp[] timestampValues) {
          TimestampColumnVector tcv = new TimestampColumnVector(size);

          tcv.noNulls = !nulls;
    @@ -95,10 +94,17 @@ public class VectorizedRowGroupGenUtil {
            if(nulls && (repeating || i % nullFrequency == 0)) {
              tcv.isNull[i] = true;
              tcv.setNullValue(i);
    -
    + timestampValues[i] = null;
            }else {
              tcv.isNull[i] = false;
    - tcv.set(i, repeating ? repeatingTimestamp : RandomTypeUtil.getRandTimestamp(rand));
    + if (!repeating) {
    + Timestamp randomTimestamp = RandomTypeUtil.getRandTimestamp(rand);
    + tcv.set(i, randomTimestamp);
    + timestampValues[i] = randomTimestamp;
    + } else {
    + tcv.set(i, repeatingTimestamp);
    + timestampValues[i] = repeatingTimestamp;
    + }
            }
          }
          return tcv;

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
    index c88f6d8..85923a8 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
    @@ -1946,7 +1946,7 @@ public class TestInputOutputFormat {
            long millis = (long) i * MILLIS_IN_DAY;
            millis -= LOCAL_TIMEZONE.getOffset(millis);
            assertEquals("checking timestamp " + i, millis,
    - timestampColumn.getTimestampMilliseconds(i));
    + timestampColumn.getTime(i));
          }
          assertEquals(false, reader.next(key, value));
        }

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
    index 3843c6d..1a97a6d 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
    @@ -541,7 +541,10 @@ public class TestOrcFile {
          int idx = 0;
          while (rows.hasNext()) {
            Object row = rows.next(null);
    - assertEquals(tslist.get(idx++).getNanos(), ((TimestampWritable) row).getNanos());
    + Timestamp tlistTimestamp = tslist.get(idx++);
    + if (tlistTimestamp.getNanos() != ((TimestampWritable) row).getNanos()) {
    + assertTrue(false);
    + }
          }
          assertEquals(0, writer.getSchema().getMaximumId());
          boolean[] expected = new boolean[] {false};

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
    index 4ca20c5..a82d672 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestVectorOrcFile.java
    @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration;
      import org.apache.hadoop.fs.FileSystem;
      import org.apache.hadoop.fs.Path;
      import org.apache.hadoop.hive.common.type.HiveDecimal;
    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
      import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
    @@ -527,7 +526,7 @@ public class TestVectorOrcFile {
          batch.size = tslist.size();
          for (int i=0; i < tslist.size(); ++i) {
            Timestamp ts = tslist.get(i);
    - vec.set(i, new PisaTimestamp(ts));
    + vec.set(i, ts);
          }
          writer.addRowBatch(batch);
          writer.close();
    @@ -1345,8 +1344,8 @@ public class TestVectorOrcFile {
            for (int ms = 1000; ms < 2000; ++ms) {
              TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
              timestampColVector.set(ms - 1000,
    - new PisaTimestamp(Timestamp.valueOf(year +
    - "-05-05 12:34:56." + ms)));
    + Timestamp.valueOf(year +
    + "-05-05 12:34:56." + ms));
              ((LongColumnVector) batch.cols[1]).vector[ms - 1000] =
                  new DateWritable(new Date(year - 1900, 11, 25)).getDays();
            }
    @@ -1385,7 +1384,7 @@ public class TestVectorOrcFile {
          UnionColumnVector union = (UnionColumnVector) batch.cols[1];
          if (ts != null) {
            TimestampColumnVector timestampColVector = (TimestampColumnVector) batch.cols[0];
    - timestampColVector.set(rowId, new PisaTimestamp(ts));
    + timestampColVector.set(rowId, ts);
          } else {
            batch.cols[0].isNull[rowId] = true;
            batch.cols[0].noNulls = false;
    @@ -2178,8 +2177,8 @@ public class TestVectorOrcFile {
          ((LongColumnVector) batch.cols[6]).vector[0] =
              new DateWritable(new Date(111, 6, 1)).getDays();
          ((TimestampColumnVector) batch.cols[7]).set(0,
    - new PisaTimestamp(new Timestamp(115, 9, 23, 10, 11, 59,
    - 999999999)));
    + new Timestamp(115, 9, 23, 10, 11, 59,
    + 999999999));
          ((DecimalColumnVector) batch.cols[8]).vector[0] =
              new HiveDecimalWritable("1.234567");
          ((BytesColumnVector) batch.cols[9]).setVal(0, "Echelon".getBytes());
    @@ -2234,9 +2233,8 @@ public class TestVectorOrcFile {
                new DateWritable(new Date(111, 6, 1)).getDays() + r;

            Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999);
    - PisaTimestamp pisaTimestamp = new PisaTimestamp(ts);
    - pisaTimestamp.addSeconds(pisaTimestamp, r, pisaTimestamp);
    - ((TimestampColumnVector) batch.cols[7]).set(r, pisaTimestamp);
    + ts.setTime(ts.getTime() + r * 1000);
    + ((TimestampColumnVector) batch.cols[7]).set(r, ts);
            ((DecimalColumnVector) batch.cols[8]).vector[r] =
                new HiveDecimalWritable("1.234567");
            ((BytesColumnVector) batch.cols[9]).setVal(r,
    @@ -2378,8 +2376,10 @@ public class TestVectorOrcFile {
                row.getFieldValue(5).toString());
            assertEquals("row " + r, new Date(111, 6, 1 + r).toString(),
                row.getFieldValue(6).toString());
    + Timestamp ts = new Timestamp(115, 9, 23, 10, 11, 59, 999999999);
    + ts.setTime(ts.getTime() + r * 1000);
            assertEquals("row " + r,
    - new Timestamp(115, 9, 23, 10, 11, 59 + r, 999999999).toString(),
    + ts.toString(),
                row.getFieldValue(7).toString());
            assertEquals("row " + r, "1.234567", row.getFieldValue(8).toString());
            assertEquals("row " + r, Integer.toString(r),

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q b/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q
    new file mode 100644
    index 0000000..40c4c03
    --- /dev/null
    +++ b/ql/src/test/queries/clientpositive/vector_interval_arithmetic.q
    @@ -0,0 +1,174 @@
    +set hive.cli.print.header=true;
    +set hive.explain.user=false;
    +set hive.fetch.task.conversion=none;
    +
    +create table unique_timestamps (tsval timestamp) STORED AS TEXTFILE;
    +
    +LOAD DATA LOCAL INPATH '../../data/files/timestamps.txt' OVERWRITE INTO TABLE unique_timestamps;
    +
    +create table interval_arithmetic_1 (dateval date, tsval timestamp) stored as orc;
    +insert overwrite table interval_arithmetic_1
    + select cast(tsval as date), tsval from unique_timestamps;
    +
    +SET hive.vectorized.execution.enabled=true;
    +
    +-- interval year-month arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +explain
    +select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +explain
    +select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval;
    +
    +select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval;
    +
    +explain
    +select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2;
    +
    +select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2;
    +
    +
    +-- interval day-time arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +explain
    +select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval;
    +
    +explain
    +select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval;
    +
    +select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval;
    +
    +explain
    +select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2;
    +
    +select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2;
    +
    +drop table interval_arithmetic_1;
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13151 : Clean up UGI objects in FileSystem cache for transactions (Wei Zheng, reviewed by Eugene Koifman)


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

    Branch: refs/heads/llap
    Commit: f9d1b6ab77ab15b8337c17fbe38557c1f7b5ce58
    Parents: d3a5f20
    Author: Wei Zheng <weiz@apache.org>
    Authored: Thu Mar 24 17:29:59 2016 -0700
    Committer: Wei Zheng <weiz@apache.org>
    Committed: Thu Mar 24 17:29:59 2016 -0700

    ----------------------------------------------------------------------
      .../hive/hcatalog/streaming/HiveEndPoint.java | 11 +++++
      .../hadoop/hive/ql/txn/compactor/Cleaner.java | 5 +++
      .../hive/ql/txn/compactor/CompactorThread.java | 5 +++
      .../hadoop/hive/ql/txn/compactor/Initiator.java | 9 +++-
      .../hadoop/hive/ql/txn/compactor/Worker.java | 8 +++-
      .../apache/hadoop/hive/ql/TestTxnCommands2.java | 47 ++++++++++++++++++++
      6 files changed, 82 insertions(+), 3 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/f9d1b6ab/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
    ----------------------------------------------------------------------
    diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
    index 4c77842..baeafad 100644
    --- a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
    +++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/HiveEndPoint.java
    @@ -18,6 +18,7 @@

      package org.apache.hive.hcatalog.streaming;

    +import org.apache.hadoop.fs.FileSystem;
      import org.slf4j.Logger;
      import org.slf4j.LoggerFactory;
      import org.apache.hadoop.hive.cli.CliSessionState;
    @@ -342,6 +343,11 @@ public class HiveEndPoint {
                      return null;
                    }
                  } );
    + try {
    + FileSystem.closeAllForUGI(ugi);
    + } catch (IOException exception) {
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception);
    + }
            } catch (IOException e) {
              LOG.error("Error closing connection to " + endPt, e);
            } catch (InterruptedException e) {
    @@ -937,6 +943,11 @@ public class HiveEndPoint {
                        }
                      }
              );
    + try {
    + FileSystem.closeAllForUGI(ugi);
    + } catch (IOException exception) {
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception);
    + }
            } catch (IOException e) {
              throw new ImpersonationFailed("Failed closing Txn Batch as user '" + username +
                      "' on endPoint :" + endPt, e);

    http://git-wip-us.apache.org/repos/asf/hive/blob/f9d1b6ab/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    index 9ffeaec..4c31a49 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    @@ -272,6 +272,11 @@ public class Cleaner extends CompactorThread {
                  return null;
                }
              });
    + try {
    + FileSystem.closeAllForUGI(ugi);
    + } catch (IOException exception) {
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception + " for " +
    + ci.getFullPartitionName()); }
            }
            txnHandler.markCleaned(ci);
          } catch (Exception e) {

    http://git-wip-us.apache.org/repos/asf/hive/blob/f9d1b6ab/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
    index 8495c66..4d6e24e 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorThread.java
    @@ -174,6 +174,11 @@ abstract class CompactorThread extends Thread implements MetaStoreThread {
                return null;
              }
            });
    + try {
    + FileSystem.closeAllForUGI(ugi);
    + } catch (IOException exception) {
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception);
    + }

            if (wrapper.size() == 1) {
              LOG.debug("Running job as " + wrapper.get(0));

    http://git-wip-us.apache.org/repos/asf/hive/blob/f9d1b6ab/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    index 916d9dc..98ebf53 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    @@ -226,12 +226,19 @@ public class Initiator extends CompactorThread {
            LOG.info("Going to initiate as user " + runAs);
            UserGroupInformation ugi = UserGroupInformation.createProxyUser(runAs,
              UserGroupInformation.getLoginUser());
    - return ugi.doAs(new PrivilegedExceptionAction<CompactionType>() {
    + CompactionType compactionType = ugi.doAs(new PrivilegedExceptionAction<CompactionType>() {
              @Override
              public CompactionType run() throws Exception {
                return determineCompactionType(ci, txns, sd);
              }
            });
    + try {
    + FileSystem.closeAllForUGI(ugi);
    + } catch (IOException exception) {
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception + " for " +
    + ci.getFullPartitionName());
    + }
    + return compactionType;
          }
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/f9d1b6ab/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    index adffa8c..e21ca27 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    @@ -17,6 +17,7 @@
       */
      package org.apache.hadoop.hive.ql.txn.compactor;

    +import org.apache.hadoop.fs.FileSystem;
      import org.slf4j.Logger;
      import org.slf4j.LoggerFactory;
      import org.apache.hadoop.hive.common.ValidTxnList;
    @@ -34,8 +35,6 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
      import org.apache.hadoop.hive.ql.session.SessionState;
      import org.apache.hadoop.security.UserGroupInformation;
      import org.apache.hadoop.util.StringUtils;
    -import org.slf4j.Logger;
    -import org.slf4j.LoggerFactory;

      import java.io.IOException;
      import java.net.InetAddress;
    @@ -173,6 +172,11 @@ public class Worker extends CompactorThread {
                      return null;
                    }
                  });
    + try {
    + FileSystem.closeAllForUGI(ugi);
    + } catch (IOException exception) {
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception + " for " +
    + ci.getFullPartitionName()); }
                }
                txnHandler.markCompacted(ci);
              } catch (Exception e) {

    http://git-wip-us.apache.org/repos/asf/hive/blob/f9d1b6ab/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
    index 0786c21..04c1d17 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands2.java
    @@ -51,10 +51,13 @@ import org.junit.Test;
      import org.junit.rules.TestName;

      import java.io.File;
    +import java.lang.reflect.Field;
      import java.util.ArrayList;
      import java.util.Arrays;
      import java.util.Comparator;
    +import java.util.HashMap;
      import java.util.List;
    +import java.util.Map;
      import java.util.concurrent.TimeUnit;
      import java.util.concurrent.atomic.AtomicBoolean;

    @@ -546,6 +549,50 @@ public class TestTxnCommands2 {
          Assert.assertEquals("Unexpected num succeeded", 1, cbs.succeeded);
          Assert.assertEquals("Unexpected num total5", hiveConf.getIntVar(HiveConf.ConfVars.COMPACTOR_HISTORY_RETENTION_FAILED) + 1, cbs.total);
        }
    +
    + /**
    + * Make sure there's no FileSystem$Cache$Key leak due to UGI use
    + * @throws Exception
    + */
    + @Test
    + public void testFileSystemUnCaching() throws Exception {
    + int cacheSizeBefore;
    + int cacheSizeAfter;
    +
    + // get the size of cache BEFORE
    + cacheSizeBefore = getFileSystemCacheSize();
    +
    + // Insert a row to ACID table
    + runStatementOnDriver("insert into " + Table.ACIDTBL + "(a,b) values(1,2)");
    +
    + // Perform a major compaction
    + runStatementOnDriver("alter table " + Table.ACIDTBL + " compact 'major'");
    + runWorker(hiveConf);
    + runCleaner(hiveConf);
    +
    + // get the size of cache AFTER
    + cacheSizeAfter = getFileSystemCacheSize();
    +
    + Assert.assertEquals(cacheSizeBefore, cacheSizeAfter);
    + }
    +
    + private int getFileSystemCacheSize() throws Exception {
    + try {
    + Field cache = FileSystem.class.getDeclaredField("CACHE");
    + cache.setAccessible(true);
    + Object o = cache.get(null); // FileSystem.CACHE
    +
    + Field mapField = o.getClass().getDeclaredField("map");
    + mapField.setAccessible(true);
    + Map map = (HashMap)mapField.get(o); // FileSystem.CACHE.map
    +
    + return map.size();
    + } catch (NoSuchFieldException e) {
    + System.out.println(e);
    + }
    + return 0;
    + }
    +
        private static class CompactionsByState {
          private int attempted;
          private int failed;
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
    index 29b51ec..f4cfa53 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReaderImpl.java
    @@ -42,7 +42,6 @@ import org.apache.hadoop.hive.ql.io.orc.RecordReaderUtils;
      import org.apache.orc.impl.StreamName;
      import org.apache.orc.StripeInformation;
      import org.apache.orc.impl.BufferChunk;
    -import org.apache.hadoop.hive.ql.io.orc.RecordReaderUtils.ByteBufferAllocatorPool;
      import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.OrcEncodedColumnBatch;
      import org.apache.hadoop.hive.ql.io.orc.encoded.Reader.PoolFactory;
      import org.apache.orc.OrcProto;
    @@ -103,8 +102,7 @@ class EncodedReaderImpl implements EncodedReader {
        private final List<OrcProto.Type> types;
        private final long rowIndexStride;
        private final DataCache cache;
    - private ByteBufferAllocatorPool pool;
    - private boolean isDebugTracingEnabled;
    + private boolean isTracingEnabled;

        public EncodedReaderImpl(Object fileKey, List<OrcProto.Type> types, CompressionCodec codec,
            int bufferSize, long strideRate, DataCache cache, DataReader dataReader, PoolFactory pf)
    @@ -209,8 +207,8 @@ class EncodedReaderImpl implements EncodedReader {
          long offset = 0; // Stream offset in relation to the stripe.
          // 1.1. Figure out which columns have a present stream
          boolean[] hasNull = RecordReaderUtils.findPresentStreamsByColumn(streamList, types);
    - if (isDebugTracingEnabled) {
    - LOG.info("The following columns have PRESENT streams: " + arrayToString(hasNull));
    + if (isTracingEnabled) {
    + LOG.trace("The following columns have PRESENT streams: " + arrayToString(hasNull));
          }

          // We assume stream list is sorted by column and that non-data
    @@ -230,8 +228,8 @@ class EncodedReaderImpl implements EncodedReader {
              // We have a stream for included column, but in future it might have no data streams.
              // It's more like "has at least one column included that has an index stream".
              hasIndexOnlyCols = hasIndexOnlyCols | included[colIx];
    - if (isDebugTracingEnabled) {
    - LOG.info("Skipping stream: " + streamKind + " at " + offset + ", " + length);
    + if (isTracingEnabled) {
    + LOG.trace("Skipping stream: " + streamKind + " at " + offset + ", " + length);
              }
              offset += length;
              continue;
    @@ -244,8 +242,8 @@ class EncodedReaderImpl implements EncodedReader {
              includedRgs = colRgs[colRgIx];
              ctx = colCtxs[colRgIx] = new ColumnReadContext(
                  colIx, encodings.get(colIx), indexes[colIx]);
    - if (isDebugTracingEnabled) {
    - LOG.info("Creating context " + colRgIx + " for column " + colIx + ":" + ctx.toString());
    + if (isTracingEnabled) {
    + LOG.trace("Creating context " + colRgIx + " for column " + colIx + ":" + ctx.toString());
              }
            } else {
              ctx = colCtxs[colRgIx];
    @@ -254,14 +252,14 @@ class EncodedReaderImpl implements EncodedReader {
            int indexIx = RecordReaderUtils.getIndexPosition(ctx.encoding.getKind(),
                types.get(colIx).getKind(), streamKind, isCompressed, hasNull[colIx]);
            ctx.addStream(offset, stream, indexIx);
    - if (isDebugTracingEnabled) {
    - LOG.info("Adding stream for column " + colIx + ": " + streamKind + " at " + offset
    + if (isTracingEnabled) {
    + LOG.trace("Adding stream for column " + colIx + ": " + streamKind + " at " + offset
                  + ", " + length + ", index position " + indexIx);
            }
            if (includedRgs == null || RecordReaderUtils.isDictionary(streamKind, encodings.get(colIx))) {
              RecordReaderUtils.addEntireStreamToRanges(offset, length, listToRead, true);
    - if (isDebugTracingEnabled) {
    - LOG.info("Will read whole stream " + streamKind + "; added to " + listToRead.getTail());
    + if (isTracingEnabled) {
    + LOG.trace("Will read whole stream " + streamKind + "; added to " + listToRead.getTail());
              }
            } else {
              RecordReaderUtils.addRgFilteredStreamToRanges(stream, includedRgs,
    @@ -287,15 +285,15 @@ class EncodedReaderImpl implements EncodedReader {

          // 2. Now, read all of the ranges from cache or disk.
          DiskRangeList.MutateHelper toRead = new DiskRangeList.MutateHelper(listToRead.get());
    - if (isDebugTracingEnabled && LOG.isInfoEnabled()) {
    - LOG.info("Resulting disk ranges to read (file " + fileKey + "): "
    + if (isTracingEnabled && LOG.isInfoEnabled()) {
    + LOG.trace("Resulting disk ranges to read (file " + fileKey + "): "
                + RecordReaderUtils.stringifyDiskRanges(toRead.next));
          }
          BooleanRef isAllInCache = new BooleanRef();
          if (hasFileId) {
            cache.getFileData(fileKey, toRead.next, stripeOffset, CC_FACTORY, isAllInCache);
    - if (isDebugTracingEnabled && LOG.isInfoEnabled()) {
    - LOG.info("Disk ranges after cache (file " + fileKey + ", base offset " + stripeOffset
    + if (isTracingEnabled && LOG.isInfoEnabled()) {
    + LOG.trace("Disk ranges after cache (file " + fileKey + ", base offset " + stripeOffset
                  + "): " + RecordReaderUtils.stringifyDiskRanges(toRead.next));
            }
          }
    @@ -322,8 +320,8 @@ class EncodedReaderImpl implements EncodedReader {
                }
              }
            }
    - if (isDebugTracingEnabled) {
    - LOG.info("Disk ranges after pre-read (file " + fileKey + ", base offset "
    + if (isTracingEnabled) {
    + LOG.trace("Disk ranges after pre-read (file " + fileKey + ", base offset "
                  + stripeOffset + "): " + RecordReaderUtils.stringifyDiskRanges(toRead.next));
            }
            iter = toRead.next; // Reset the iter to start.
    @@ -354,8 +352,8 @@ class EncodedReaderImpl implements EncodedReader {
                ColumnStreamData cb = null;
                if (RecordReaderUtils.isDictionary(sctx.kind, ctx.encoding)) {
                  // This stream is for entire stripe and needed for every RG; uncompress once and reuse.
    - if (isDebugTracingEnabled) {
    - LOG.info("Getting stripe-level stream [" + sctx.kind + ", " + ctx.encoding + "] for"
    + if (isTracingEnabled) {
    + LOG.trace("Getting stripe-level stream [" + sctx.kind + ", " + ctx.encoding + "] for"
                        + " column " + ctx.colIx + " RG " + rgIx + " at " + sctx.offset + ", " + sctx.length);
                  }
                  if (sctx.stripeLevelStream == null) {
    @@ -411,8 +409,8 @@ class EncodedReaderImpl implements EncodedReader {
            }
          }

    - if (isDebugTracingEnabled) {
    - LOG.info("Disk ranges after preparing all the data "
    + if (isTracingEnabled) {
    + LOG.trace("Disk ranges after preparing all the data "
                + RecordReaderUtils.stringifyDiskRanges(toRead.next));
          }

    @@ -437,8 +435,8 @@ class EncodedReaderImpl implements EncodedReader {
            int colIx, StreamContext sctx, long cOffset, long endCOffset, boolean isCompressed) {
          ColumnStreamData cb = POOLS.csdPool.take();
          cb.incRef();
    - if (isDebugTracingEnabled) {
    - LOG.info("Getting data for column "+ colIx + " " + (isLastRg ? "last " : "")
    + if (isTracingEnabled) {
    + LOG.trace("Getting data for column "+ colIx + " " + (isLastRg ? "last " : "")
                + "RG " + rgIx + " stream " + sctx.kind + " at " + sctx.offset + ", "
                + sctx.length + " index position " + sctx.streamIndexOffset + ": " +
                (isCompressed ? "" : "un") + "compressed [" + cOffset + ", " + endCOffset + ")");
    @@ -460,17 +458,14 @@ class EncodedReaderImpl implements EncodedReader {
        }

        @Override
    - public void setDebugTracing(boolean isEnabled) {
    - this.isDebugTracingEnabled = isEnabled;
    + public void setTracing(boolean isEnabled) {
    + this.isTracingEnabled = isEnabled;
        }


        @Override
        public void close() throws IOException {
          dataReader.close();
    - if (pool != null) {
    - pool.clear();
    - }
        }

        /**
    @@ -608,8 +603,8 @@ class EncodedReaderImpl implements EncodedReader {
          // want to be, or just before. However, RGs can overlap due to encoding, so we may have
          // to return to a previous block.
          DiskRangeList current = findExactPosition(start, cOffset);
    - if (isDebugTracingEnabled) {
    - LOG.info("Starting read for [" + cOffset + "," + endCOffset + ") at " + current);
    + if (isTracingEnabled) {
    + LOG.trace("Starting read for [" + cOffset + "," + endCOffset + ") at " + current);
          }

          CacheChunk lastUncompressed = null;
    @@ -648,8 +643,8 @@ class EncodedReaderImpl implements EncodedReader {
            }

            chunk.originalData = null;
    - if (isDebugTracingEnabled) {
    - LOG.info("Locking " + chunk.getBuffer() + " due to reuse (after decompression)");
    + if (isTracingEnabled) {
    + LOG.trace("Locking " + chunk.getBuffer() + " due to reuse (after decompression)");
            }
            cache.reuseBuffer(chunk.getBuffer());
          }
    @@ -691,22 +686,22 @@ class EncodedReaderImpl implements EncodedReader {
            if (current instanceof CacheChunk) {
              // 2a. This is a decoded compression buffer, add as is.
              CacheChunk cc = (CacheChunk)current;
    - if (isDebugTracingEnabled) {
    - LOG.info("Locking " + cc.getBuffer() + " due to reuse");
    + if (isTracingEnabled) {
    + LOG.trace("Locking " + cc.getBuffer() + " due to reuse");
              }
              cache.reuseBuffer(cc.getBuffer());
              columnStreamData.getCacheBuffers().add(cc.getBuffer());
              currentOffset = cc.getEnd();
    - if (isDebugTracingEnabled) {
    - LOG.info("Adding an already-uncompressed buffer " + cc.getBuffer());
    + if (isTracingEnabled) {
    + LOG.trace("Adding an already-uncompressed buffer " + cc.getBuffer());
              }
              ponderReleaseInitialRefcount(unlockUntilCOffset, streamOffset, cc);
              lastUncompressed = cc;
              next = current.next;
            } else if (current instanceof IncompleteCb) {
              // 2b. This is a known incomplete CB caused by ORC CB end boundaries being estimates.
    - if (isDebugTracingEnabled) {
    - LOG.info("Cannot read " + current);
    + if (isTracingEnabled) {
    + LOG.trace("Cannot read " + current);
              }
              next = null;
              currentOffset = -1;
    @@ -739,8 +734,8 @@ class EncodedReaderImpl implements EncodedReader {
            DiskRangeList next = null;
            assert current instanceof CacheChunk;
            lastUncompressed = (CacheChunk)current;
    - if (isDebugTracingEnabled) {
    - LOG.info("Locking " + lastUncompressed.getBuffer() + " due to reuse");
    + if (isTracingEnabled) {
    + LOG.trace("Locking " + lastUncompressed.getBuffer() + " due to reuse");
            }
            cache.reuseBuffer(lastUncompressed.getBuffer());
            if (isFirst) {
    @@ -749,8 +744,8 @@ class EncodedReaderImpl implements EncodedReader {
            }
            columnStreamData.getCacheBuffers().add(lastUncompressed.getBuffer());
            currentOffset = lastUncompressed.getEnd();
    - if (isDebugTracingEnabled) {
    - LOG.info("Adding an uncompressed buffer " + lastUncompressed.getBuffer());
    + if (isTracingEnabled) {
    + LOG.trace("Adding an uncompressed buffer " + lastUncompressed.getBuffer());
            }
            ponderReleaseInitialRefcount(unlockUntilCOffset, streamOffset, lastUncompressed);
            next = current.next;
    @@ -770,7 +765,6 @@ class EncodedReaderImpl implements EncodedReader {
         * to handle just for this case.
         * We could avoid copy in non-zcr case and manage the buffer that was not allocated by our
         * allocator. Uncompressed case is not mainline though so let's not complicate it.
    - * @param qfCounters
         */
        private DiskRangeList preReadUncompressedStream(long baseOffset,
            DiskRangeList start, long streamOffset, long streamEnd) throws IOException {
    @@ -780,8 +774,8 @@ class EncodedReaderImpl implements EncodedReader {

          // 1. Find our bearings in the stream.
          DiskRangeList current = findIntersectingPosition(start, streamOffset, streamEnd);
    - if (isDebugTracingEnabled) {
    - LOG.info("Starting pre-read for [" + streamOffset + "," + streamEnd + ") at " + current);
    + if (isTracingEnabled) {
    + LOG.trace("Starting pre-read for [" + streamOffset + "," + streamEnd + ") at " + current);
          }

          if (streamOffset > current.getOffset()) {
    @@ -836,8 +830,8 @@ class EncodedReaderImpl implements EncodedReader {
                current = current.split(partEnd);
                wasSplit = true;
              }
    - if (isDebugTracingEnabled) {
    - LOG.info("Processing uncompressed file data at ["
    + if (isTracingEnabled) {
    + LOG.trace("Processing uncompressed file data at ["
                    + current.getOffset() + ", " + current.getEnd() + ")");
              }
              BufferChunk curBc = (BufferChunk)current;
    @@ -1058,8 +1052,8 @@ class EncodedReaderImpl implements EncodedReader {

        private void releaseInitialRefcount(CacheChunk cc, boolean isBacktracking) {
          // This is the last RG for which this buffer will be used. Remove the initial refcount
    - if (isDebugTracingEnabled) {
    - LOG.info("Unlocking " + cc.getBuffer() + " for the fetching thread"
    + if (isTracingEnabled) {
    + LOG.trace("Unlocking " + cc.getBuffer() + " for the fetching thread"
                + (isBacktracking ? "; backtracking" : ""));
          }
          cache.releaseBuffer(cc.getBuffer());
    @@ -1081,8 +1075,8 @@ class EncodedReaderImpl implements EncodedReader {
              // Cache has found an old buffer for the key and put it into array instead of our new one.
              CacheChunk replacedChunk = toDecompress.get(i);
              MemoryBuffer replacementBuffer = targetBuffers[i];
    - if (isDebugTracingEnabled) {
    - LOG.info("Discarding data due to cache collision: " + replacedChunk.getBuffer()
    + if (isTracingEnabled) {
    + LOG.trace("Discarding data due to cache collision: " + replacedChunk.getBuffer()
                    + " replaced with " + replacementBuffer);
              }
              assert replacedChunk.getBuffer() != replacementBuffer : i + " was not replaced in the results "
    @@ -1133,7 +1127,6 @@ class EncodedReaderImpl implements EncodedReader {
         * multiple ranges (usually, that would only happen with zcr).
         * Adds stuff to cachedBuffers, toDecompress and toRelease (see below what each does).
         * @param current BufferChunk where compression block starts.
    - * @param ranges Iterator of all chunks, pointing at current.
         * @param cacheBuffers The result buffer array to add pre-allocated target cache buffer.
         * @param toDecompress The list of work to decompress - pairs of compressed buffers and the
         * target buffers (same as the ones added to cacheBuffers).
    @@ -1157,8 +1150,8 @@ class EncodedReaderImpl implements EncodedReader {
          int consumedLength = chunkLength + OutStream.HEADER_SIZE;
          long cbEndOffset = cbStartOffset + consumedLength;
          boolean isUncompressed = ((b0 & 0x01) == 1);
    - if (isDebugTracingEnabled) {
    - LOG.info("Found CB at " + cbStartOffset + ", chunk length " + chunkLength + ", total "
    + if (isTracingEnabled) {
    + LOG.trace("Found CB at " + cbStartOffset + ", chunk length " + chunkLength + ", total "
                + consumedLength + ", " + (isUncompressed ? "not " : "") + "compressed");
          }
          if (compressed.remaining() >= chunkLength) {
    @@ -1183,8 +1176,8 @@ class EncodedReaderImpl implements EncodedReader {
          int remaining = chunkLength - compressed.remaining();
          int originalPos = compressed.position();
          copy.put(compressed);
    - if (isDebugTracingEnabled) {
    - LOG.info("Removing partial CB " + current + " from ranges after copying its contents");
    + if (isTracingEnabled) {
    + LOG.trace("Removing partial CB " + current + " from ranges after copying its contents");
          }
          DiskRangeList next = current.next;
          current.removeSelf();
    @@ -1223,8 +1216,8 @@ class EncodedReaderImpl implements EncodedReader {
            DiskRangeList tmp = next;
            next = next.hasContiguousNext() ? next.next : null;
            if (next != null) {
    - if (isDebugTracingEnabled) {
    - LOG.info("Removing partial CB " + tmp + " from ranges after copying its contents");
    + if (isTracingEnabled) {
    + LOG.trace("Removing partial CB " + tmp + " from ranges after copying its contents");
              }
              tmp.removeSelf();
            } else {
    @@ -1237,8 +1230,8 @@ class EncodedReaderImpl implements EncodedReader {
        private void addIncompleteCompressionBuffer(
            long cbStartOffset, DiskRangeList target, int extraChunkCount) {
          IncompleteCb icb = new IncompleteCb(cbStartOffset, target.getEnd());
    - if (isDebugTracingEnabled) {
    - LOG.info("Replacing " + target + " (and " + extraChunkCount + " previous chunks) with "
    + if (isTracingEnabled) {
    + LOG.trace("Replacing " + target + " (and " + extraChunkCount + " previous chunks) with "
                + icb + " in the buffers");
          }
          target.replaceSelfWith(icb);
    @@ -1250,9 +1243,7 @@ class EncodedReaderImpl implements EncodedReader {
         * @param isUncompressed Whether the data in the block is uncompressed.
         * @param cbStartOffset Compressed start offset of the fCB.
         * @param cbEndOffset Compressed end offset of the fCB.
    - * @param lastRange The buffer from which the last (or all) bytes of fCB come.
         * @param lastChunkLength The number of compressed bytes consumed from last *chunk* into fullCompressionBlock.
    - * @param ranges The iterator of all compressed ranges for the stream, pointing at lastRange.
         * @param lastChunk
         * @param toDecompress See addOneCompressionBuffer.
         * @param cacheBuffers See addOneCompressionBuffer.
    @@ -1271,20 +1262,20 @@ class EncodedReaderImpl implements EncodedReader {
              fullCompressionBlock, futureAlloc, cacheBuffers.size() - 1);
          toDecompress.add(cc);
          // Adjust the compression block position.
    - if (isDebugTracingEnabled) {
    - LOG.info("Adjusting " + lastChunk + " to consume " + lastChunkLength + " compressed bytes");
    + if (isTracingEnabled) {
    + LOG.trace("Adjusting " + lastChunk + " to consume " + lastChunkLength + " compressed bytes");
          }
          lastChunk.getChunk().position(lastChunk.getChunk().position() + lastChunkLength);
          // Finally, put it in the ranges list for future use (if shared between RGs).
          // Before anyone else accesses it, it would have been allocated and decompressed locally.
          if (lastChunk.getChunk().remaining() <= 0) {
    - if (isDebugTracingEnabled) {
    - LOG.info("Replacing " + lastChunk + " with " + cc + " in the buffers");
    + if (isTracingEnabled) {
    + LOG.trace("Replacing " + lastChunk + " with " + cc + " in the buffers");
            }
            lastChunk.replaceSelfWith(cc);
          } else {
    - if (isDebugTracingEnabled) {
    - LOG.info("Adding " + cc + " before " + lastChunk + " in the buffers");
    + if (isTracingEnabled) {
    + LOG.trace("Adding " + cc + " before " + lastChunk + " in the buffers");
            }
            lastChunk.insertPartBefore(cc);
          }
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12367 : Lock/unlock database should add current database to inputs and outputs of authz hook (Dapeng Sun via Ashutosh Chauhan)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6bfec2e9
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6bfec2e9
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6bfec2e9

    Branch: refs/heads/llap
    Commit: 6bfec2e97c4e434646aa9aeffd98c9939313fa6e
    Parents: a71edcf
    Author: Dapeng Sun <sdp@apache.org>
    Authored: Tue Dec 15 19:39:00 2015 -0800
    Committer: Ashutosh Chauhan <hashutosh@apache.org>
    Committed: Thu Mar 24 19:22:28 2016 -0700

    ----------------------------------------------------------------------
      .../java/org/apache/hadoop/hive/ql/Driver.java | 20 ++++++++++++++++++++
      .../hive/ql/parse/DDLSemanticAnalyzer.java | 15 +++++++++++++--
      .../clientnegative/dbtxnmgr_nodblock.q.out | 2 ++
      .../clientnegative/dbtxnmgr_nodbunlock.q.out | 2 ++
      .../lockneg_query_tbl_in_locked_db.q.out | 6 ++++++
      .../lockneg_try_db_lock_conflict.q.out | 6 ++++++
      .../lockneg_try_drop_locked_db.q.out | 4 ++++
      .../lockneg_try_lock_db_in_use.q.out | 6 ++++++
      8 files changed, 59 insertions(+), 2 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    index d7e4ac7..7276e31 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
    @@ -1416,6 +1416,10 @@ public class Driver implements CommandProcessor {
          if (!checkConcurrency()) {
            return false;
          }
    + // Lock operations themselves don't require the lock.
    + if (isExplicitLockOperation()){
    + return false;
    + }
          if (!HiveConf.getBoolVar(conf, ConfVars.HIVE_LOCK_MAPRED_ONLY)) {
            return true;
          }
    @@ -1438,6 +1442,22 @@ public class Driver implements CommandProcessor {
          return false;
        }

    + private boolean isExplicitLockOperation() {
    + HiveOperation currentOpt = plan.getOperation();
    + if (currentOpt != null) {
    + switch (currentOpt) {
    + case LOCKDB:
    + case UNLOCKDB:
    + case LOCKTABLE:
    + case UNLOCKTABLE:
    + return true;
    + default:
    + return false;
    + }
    + }
    + return false;
    + }
    +
        private CommandProcessorResponse createProcessorResponse(int ret) {
          queryDisplay.setErrorMessage(errorMessage);
          return new CommandProcessorResponse(ret, errorMessage, SQLState, downstreamError);

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    index 0c087ed..fe9b8cc 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    @@ -2444,8 +2444,12 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
          String dbName = unescapeIdentifier(ast.getChild(0).getText());
          String mode = unescapeIdentifier(ast.getChild(1).getText().toUpperCase());

    - //inputs.add(new ReadEntity(dbName));
    - //outputs.add(new WriteEntity(dbName));
    + inputs.add(new ReadEntity(getDatabase(dbName)));
    + // Lock database operation is to acquire the lock explicitly, the operation
    + // itself doesn't need to be locked. Set the WriteEntity as WriteType:
    + // DDL_NO_LOCK here, otherwise it will conflict with Hive's transaction.
    + outputs.add(new WriteEntity(getDatabase(dbName), WriteType.DDL_NO_LOCK));
    +
          LockDatabaseDesc lockDatabaseDesc = new LockDatabaseDesc(dbName, mode,
                              HiveConf.getVar(conf, ConfVars.HIVEQUERYID));
          lockDatabaseDesc.setQueryStr(ctx.getCmd());
    @@ -2457,6 +2461,13 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
        private void analyzeUnlockDatabase(ASTNode ast) throws SemanticException {
          String dbName = unescapeIdentifier(ast.getChild(0).getText());

    + inputs.add(new ReadEntity(getDatabase(dbName)));
    + // Unlock database operation is to release the lock explicitly, the
    + // operation itself don't need to be locked. Set the WriteEntity as
    + // WriteType: DDL_NO_LOCK here, otherwise it will conflict with
    + // Hive's transaction.
    + outputs.add(new WriteEntity(getDatabase(dbName), WriteType.DDL_NO_LOCK));
    +
          UnlockDatabaseDesc unlockDatabaseDesc = new UnlockDatabaseDesc(dbName);
          DDLWork work = new DDLWork(getInputs(), getOutputs(), unlockDatabaseDesc);
          rootTasks.add(TaskFactory.get(work, conf));

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out b/ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out
    index b8bbe93..e783251 100644
    --- a/ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out
    +++ b/ql/src/test/results/clientnegative/dbtxnmgr_nodblock.q.out
    @@ -10,4 +10,6 @@ POSTHOOK: type: CREATEDATABASE
      POSTHOOK: Output: database:drop_nodblock
      PREHOOK: query: lock database drop_nodblock shared
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:drop_nodblock
    +PREHOOK: Output: database:drop_nodblock
      FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Current transaction manager does not support explicit lock requests. Transaction manager: org.apache.hadoop.hive.ql.lockmgr.DbTxnManager

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out b/ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out
    index ea7a3eb..d7a39f0 100644
    --- a/ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out
    +++ b/ql/src/test/results/clientnegative/dbtxnmgr_nodbunlock.q.out
    @@ -10,4 +10,6 @@ POSTHOOK: type: CREATEDATABASE
      POSTHOOK: Output: database:drop_nodbunlock
      PREHOOK: query: unlock database drop_nodbunlock
      PREHOOK: type: UNLOCKDATABASE
    +PREHOOK: Input: database:drop_nodbunlock
    +PREHOOK: Output: database:drop_nodbunlock
      FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Current transaction manager does not support explicit lock requests. Transaction manager: org.apache.hadoop.hive.ql.lockmgr.DbTxnManager

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/test/results/clientnegative/lockneg_query_tbl_in_locked_db.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientnegative/lockneg_query_tbl_in_locked_db.q.out b/ql/src/test/results/clientnegative/lockneg_query_tbl_in_locked_db.q.out
    index 134a57b..2c46159 100644
    --- a/ql/src/test/results/clientnegative/lockneg_query_tbl_in_locked_db.q.out
    +++ b/ql/src/test/results/clientnegative/lockneg_query_tbl_in_locked_db.q.out
    @@ -34,8 +34,12 @@ POSTHOOK: Lineage: tstsrcpart PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpar
      POSTHOOK: Lineage: tstsrcpart PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
      PREHOOK: query: lock database lockneg1 shared
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:lockneg1
    +PREHOOK: Output: database:lockneg1
      POSTHOOK: query: lock database lockneg1 shared
      POSTHOOK: type: LOCKDATABASE
    +POSTHOOK: Input: database:lockneg1
    +POSTHOOK: Output: database:lockneg1
      PREHOOK: query: show locks database lockneg1
      PREHOOK: type: SHOWLOCKS
      POSTHOOK: query: show locks database lockneg1
    @@ -53,4 +57,6 @@ POSTHOOK: Input: lockneg1@tstsrcpart@ds=2008-04-08/hr=11
      500
      PREHOOK: query: unlock database lockneg1
      PREHOOK: type: UNLOCKDATABASE
    +PREHOOK: Input: database:lockneg1
    +PREHOOK: Output: database:lockneg1
      FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Database lockneg1 is not locked

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/test/results/clientnegative/lockneg_try_db_lock_conflict.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientnegative/lockneg_try_db_lock_conflict.q.out b/ql/src/test/results/clientnegative/lockneg_try_db_lock_conflict.q.out
    index 0a5d98a..fe12d83 100644
    --- a/ql/src/test/results/clientnegative/lockneg_try_db_lock_conflict.q.out
    +++ b/ql/src/test/results/clientnegative/lockneg_try_db_lock_conflict.q.out
    @@ -6,9 +6,15 @@ POSTHOOK: type: CREATEDATABASE
      POSTHOOK: Output: database:lockneg4
      PREHOOK: query: lock database lockneg4 exclusive
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:lockneg4
    +PREHOOK: Output: database:lockneg4
      POSTHOOK: query: lock database lockneg4 exclusive
      POSTHOOK: type: LOCKDATABASE
    +POSTHOOK: Input: database:lockneg4
    +POSTHOOK: Output: database:lockneg4
      PREHOOK: query: lock database lockneg4 shared
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:lockneg4
    +PREHOOK: Output: database:lockneg4
      Unable to acquire EXPLICIT, SHARED lock lockneg4 after 1 attempts.
      FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/test/results/clientnegative/lockneg_try_drop_locked_db.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientnegative/lockneg_try_drop_locked_db.q.out b/ql/src/test/results/clientnegative/lockneg_try_drop_locked_db.q.out
    index 5ac4e3c..e66965e 100644
    --- a/ql/src/test/results/clientnegative/lockneg_try_drop_locked_db.q.out
    +++ b/ql/src/test/results/clientnegative/lockneg_try_drop_locked_db.q.out
    @@ -6,8 +6,12 @@ POSTHOOK: type: CREATEDATABASE
      POSTHOOK: Output: database:lockneg9
      PREHOOK: query: lock database lockneg9 shared
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:lockneg9
    +PREHOOK: Output: database:lockneg9
      POSTHOOK: query: lock database lockneg9 shared
      POSTHOOK: type: LOCKDATABASE
    +POSTHOOK: Input: database:lockneg9
    +POSTHOOK: Output: database:lockneg9
      PREHOOK: query: show locks database lockneg9
      PREHOOK: type: SHOWLOCKS
      POSTHOOK: query: show locks database lockneg9

    http://git-wip-us.apache.org/repos/asf/hive/blob/6bfec2e9/ql/src/test/results/clientnegative/lockneg_try_lock_db_in_use.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientnegative/lockneg_try_lock_db_in_use.q.out b/ql/src/test/results/clientnegative/lockneg_try_lock_db_in_use.q.out
    index 5486151..e5c8f3e 100644
    --- a/ql/src/test/results/clientnegative/lockneg_try_lock_db_in_use.q.out
    +++ b/ql/src/test/results/clientnegative/lockneg_try_lock_db_in_use.q.out
    @@ -34,13 +34,19 @@ POSTHOOK: Lineage: tstsrcpart PARTITION(ds=2008-04-08,hr=11).key SIMPLE [(srcpar
      POSTHOOK: Lineage: tstsrcpart PARTITION(ds=2008-04-08,hr=11).value SIMPLE [(srcpart)srcpart.FieldSchema(name:value, type:string, comment:default), ]
      PREHOOK: query: lock database lockneg2 shared
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:lockneg2
    +PREHOOK: Output: database:lockneg2
      POSTHOOK: query: lock database lockneg2 shared
      POSTHOOK: type: LOCKDATABASE
    +POSTHOOK: Input: database:lockneg2
    +POSTHOOK: Output: database:lockneg2
      PREHOOK: query: show locks
      PREHOOK: type: SHOWLOCKS
      POSTHOOK: query: show locks
      POSTHOOK: type: SHOWLOCKS
      PREHOOK: query: lock database lockneg2 exclusive
      PREHOOK: type: LOCKDATABASE
    +PREHOOK: Input: database:lockneg2
    +PREHOOK: Output: database:lockneg2
      Unable to acquire EXPLICIT, EXCLUSIVE lock lockneg2 after 1 attempts.
      FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12960: Migrate Column Stats Extrapolation and UniformDistribution to HBaseStore (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

    Branch: refs/heads/llap
    Commit: 968620932301dc64cd435292726943a6c0a42551
    Parents: 3038b05
    Author: Pengcheng Xiong <pxiong@apache.org>
    Authored: Sun Mar 27 11:46:17 2016 -0700
    Committer: Pengcheng Xiong <pxiong@apache.org>
    Committed: Sun Mar 27 12:11:39 2016 -0700

    ----------------------------------------------------------------------
      .../hive/metastore/StatObjectConverter.java | 2 +-
      .../hadoop/hive/metastore/hbase/HBaseUtils.java | 8 +-
      .../hadoop/hive/metastore/hbase/StatsCache.java | 20 +-
      .../stats/BinaryColumnStatsAggregator.java | 43 +-
      .../stats/BooleanColumnStatsAggregator.java | 42 +-
      .../hbase/stats/ColumnStatsAggregator.java | 12 +-
      .../stats/ColumnStatsAggregatorFactory.java | 8 +-
      .../stats/DecimalColumnStatsAggregator.java | 340 ++++++++-
      .../stats/DoubleColumnStatsAggregator.java | 307 +++++++-
      .../hbase/stats/IExtrapolatePartStatus.java | 30 +
      .../hbase/stats/LongColumnStatsAggregator.java | 305 +++++++-
      .../stats/StringColumnStatsAggregator.java | 85 ++-
      ...stHBaseAggregateStatsCacheWithBitVector.java | 6 +-
      .../TestHBaseAggregateStatsExtrapolation.java | 717 +++++++++++++++++++
      .../TestHBaseAggregateStatsNDVUniformDist.java | 581 +++++++++++++++
      .../clientpositive/tez/explainuser_1.q.out | 92 +--
      16 files changed, 2454 insertions(+), 144 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java b/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
    index b3ceff1..e119dd8 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/StatObjectConverter.java
    @@ -650,7 +650,7 @@ public class StatObjectConverter {
          }
        }

    - private static Decimal createThriftDecimal(String s) {
    + public static Decimal createThriftDecimal(String s) {
          BigDecimal d = new BigDecimal(s);
          return new Decimal(ByteBuffer.wrap(d.unscaledValue().toByteArray()), (short)d.scale());
        }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
    index 9ec7cd5..e0b449b 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/HBaseUtils.java
    @@ -19,6 +19,8 @@
      package org.apache.hadoop.hive.metastore.hbase;

      import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
      import java.nio.charset.Charset;
      import java.nio.charset.StandardCharsets;
      import java.security.MessageDigest;
    @@ -88,7 +90,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
      /**
       * Utility functions
       */
    -class HBaseUtils {
    +public class HBaseUtils {

        final static Charset ENCODING = StandardCharsets.UTF_8;
        final static char KEY_SEPARATOR = '\u0001';
    @@ -1421,4 +1423,8 @@ class HBaseUtils {
          b[7] = (byte)(v >>> 0);
          return b;
        }
    +
    + public static double getDoubleValue(Decimal decimal) {
    + return new BigDecimal(new BigInteger(decimal.getUnscaled()), decimal.getScale()).doubleValue();
    + }
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
    index f1d2e50..18f8afc 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/StatsCache.java
    @@ -85,12 +85,12 @@ class StatsCache {
                @Override
                public AggrStats load(StatsCacheKey key) throws Exception {
                  int numBitVectors = HiveStatsUtils.getNumBitVectorsForNDVEstimation(conf);
    + boolean useDensityFunctionForNDVEstimation = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_METASTORE_STATS_NDV_DENSITY_FUNCTION);
                  HBaseReadWrite hrw = HBaseReadWrite.getInstance();
                  AggrStats aggrStats = hrw.getAggregatedStats(key.hashed);
                  if (aggrStats == null) {
                    misses.incr();
                    ColumnStatsAggregator aggregator = null;
    - ColumnStatisticsObj statsObj = null;
                    aggrStats = new AggrStats();
                    LOG.debug("Unable to find aggregated stats for " + key.colName + ", aggregating");
                    List<ColumnStatistics> css = hrw.getPartitionStatistics(key.dbName, key.tableName,
    @@ -98,19 +98,13 @@ class StatsCache {
                        Collections.singletonList(key.colName));
                    if (css != null && css.size() > 0) {
                      aggrStats.setPartsFound(css.size());
    - for (ColumnStatistics cs : css) {
    - for (ColumnStatisticsObj cso : cs.getStatsObj()) {
    - if (statsObj == null) {
    - statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(key.colName,
    - cso.getColType(), cso.getStatsData().getSetField());
    - }
    - if (aggregator == null) {
    - aggregator = ColumnStatsAggregatorFactory.getColumnStatsAggregator(
    - cso.getStatsData().getSetField(), numBitVectors);
    - }
    - aggregator.aggregate(statsObj, cso);
    - }
    + if (aggregator == null) {
    + aggregator = ColumnStatsAggregatorFactory.getColumnStatsAggregator(css.iterator()
    + .next().getStatsObj().iterator().next().getStatsData().getSetField(),
    + numBitVectors, useDensityFunctionForNDVEstimation);
                      }
    + ColumnStatisticsObj statsObj = aggregator
    + .aggregate(key.colName, key.partNames, css);
                      aggrStats.addToColStats(statsObj);
                      me.put(key, aggrStats);
                    }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BinaryColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BinaryColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BinaryColumnStatsAggregator.java
    index 40340dd..d81d612 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BinaryColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BinaryColumnStatsAggregator.java
    @@ -19,17 +19,46 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    +import java.util.List;
    +
      import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    +import org.apache.hadoop.hive.metastore.api.MetaException;

    -public class BinaryColumnStatsAggregator extends ColumnStatsAggregator{
    +public class BinaryColumnStatsAggregator extends ColumnStatsAggregator {

        @Override
    - public void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats) {
    - BinaryColumnStatsData aggregateData = aggregateColStats.getStatsData().getBinaryStats();
    - BinaryColumnStatsData newData = newColStats.getStatsData().getBinaryStats();
    - aggregateData.setMaxColLen(Math.max(aggregateData.getMaxColLen(), newData.getMaxColLen()));
    - aggregateData.setAvgColLen(Math.max(aggregateData.getAvgColLen(), newData.getAvgColLen()));
    - aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + public ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException {
    + ColumnStatisticsObj statsObj = null;
    + BinaryColumnStatsData aggregateData = null;
    + String colType = null;
    + for (ColumnStatistics cs : css) {
    + if (cs.getStatsObjSize() != 1) {
    + throw new MetaException(
    + "The number of columns should be exactly one in aggrStats, but found "
    + + cs.getStatsObjSize());
    + }
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + if (statsObj == null) {
    + colType = cso.getColType();
    + statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType, cso
    + .getStatsData().getSetField());
    + }
    + BinaryColumnStatsData newData = cso.getStatsData().getBinaryStats();
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData.setMaxColLen(Math.max(aggregateData.getMaxColLen(), newData.getMaxColLen()));
    + aggregateData.setAvgColLen(Math.max(aggregateData.getAvgColLen(), newData.getAvgColLen()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + }
    + }
    + ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
    + columnStatisticsData.setBinaryStats(aggregateData);
    + statsObj.setStatsData(columnStatisticsData);
    + return statsObj;
        }
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BooleanColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BooleanColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BooleanColumnStatsAggregator.java
    index 735d965..e796df2 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BooleanColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/BooleanColumnStatsAggregator.java
    @@ -19,17 +19,47 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    +import java.util.List;
    +
      import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    +import org.apache.hadoop.hive.metastore.api.MetaException;

      public class BooleanColumnStatsAggregator extends ColumnStatsAggregator {

        @Override
    - public void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats) {
    - BooleanColumnStatsData aggregateData = aggregateColStats.getStatsData().getBooleanStats();
    - BooleanColumnStatsData newData = newColStats.getStatsData().getBooleanStats();
    - aggregateData.setNumTrues(aggregateData.getNumTrues() + newData.getNumTrues());
    - aggregateData.setNumFalses(aggregateData.getNumFalses() + newData.getNumFalses());
    - aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + public ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException {
    + ColumnStatisticsObj statsObj = null;
    + BooleanColumnStatsData aggregateData = null;
    + String colType = null;
    + for (ColumnStatistics cs : css) {
    + if (cs.getStatsObjSize() != 1) {
    + throw new MetaException(
    + "The number of columns should be exactly one in aggrStats, but found "
    + + cs.getStatsObjSize());
    + }
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + if (statsObj == null) {
    + colType = cso.getColType();
    + statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType, cso
    + .getStatsData().getSetField());
    + }
    + BooleanColumnStatsData newData = cso.getStatsData().getBooleanStats();
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData.setNumTrues(aggregateData.getNumTrues() + newData.getNumTrues());
    + aggregateData.setNumFalses(aggregateData.getNumFalses() + newData.getNumFalses());
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + }
    + }
    + ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
    + columnStatisticsData.setBooleanStats(aggregateData);
    + statsObj.setStatsData(columnStatisticsData);
    + return statsObj;
        }
    +
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregator.java
    index 694e53b..31955b4 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregator.java
    @@ -19,10 +19,16 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    -import org.apache.hadoop.hive.metastore.NumDistinctValueEstimator;
    +import java.util.List;
    +
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    +import org.apache.hadoop.hive.metastore.api.MetaException;

      public abstract class ColumnStatsAggregator {
    - NumDistinctValueEstimator ndvEstimator = null;
    - public abstract void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats);
    + public int numBitVectors;
    + public boolean useDensityFunctionForNDVEstimation;
    +
    + public abstract ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException;
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
    index 8eb127b..daf8569 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/ColumnStatsAggregatorFactory.java
    @@ -19,7 +19,6 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    -import org.apache.hadoop.hive.metastore.NumDistinctValueEstimator;
      import org.apache.hadoop.hive.metastore.api.BinaryColumnStatsData;
      import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
    @@ -35,7 +34,7 @@ public class ColumnStatsAggregatorFactory {
        private ColumnStatsAggregatorFactory() {
        }

    - public static ColumnStatsAggregator getColumnStatsAggregator(_Fields type, int numBitVectors) {
    + public static ColumnStatsAggregator getColumnStatsAggregator(_Fields type, int numBitVectors, boolean useDensityFunctionForNDVEstimation) {
          ColumnStatsAggregator agg;
          switch (type) {
          case BOOLEAN_STATS:
    @@ -59,9 +58,8 @@ public class ColumnStatsAggregatorFactory {
          default:
            throw new RuntimeException("Woh, bad. Unknown stats type " + type.toString());
          }
    - if (numBitVectors > 0) {
    - agg.ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    - }
    + agg.numBitVectors = numBitVectors;
    + agg.useDensityFunctionForNDVEstimation = useDensityFunctionForNDVEstimation;
          return agg;
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DecimalColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DecimalColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DecimalColumnStatsAggregator.java
    index 50f4325..36b2c9c 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DecimalColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DecimalColumnStatsAggregator.java
    @@ -19,33 +19,333 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
      import org.apache.hadoop.hive.metastore.NumDistinctValueEstimator;
    +import org.apache.hadoop.hive.metastore.StatObjectConverter;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    -import org.apache.hadoop.hive.metastore.api.Decimal;
      import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.MetaException;
    +import org.apache.hadoop.hive.metastore.hbase.HBaseUtils;

    -public class DecimalColumnStatsAggregator extends ColumnStatsAggregator {
    +public class DecimalColumnStatsAggregator extends ColumnStatsAggregator implements
    + IExtrapolatePartStatus {

        @Override
    - public void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats) {
    - DecimalColumnStatsData aggregateData = aggregateColStats.getStatsData().getDecimalStats();
    - DecimalColumnStatsData newData = newColStats.getStatsData().getDecimalStats();
    - Decimal lowValue = aggregateData.getLowValue() != null
    - && (aggregateData.getLowValue().compareTo(newData.getLowValue()) > 0) ? aggregateData
    - .getLowValue() : newData.getLowValue();
    - aggregateData.setLowValue(lowValue);
    - Decimal highValue = aggregateData.getHighValue() != null
    - && (aggregateData.getHighValue().compareTo(newData.getHighValue()) > 0) ? aggregateData
    - .getHighValue() : newData.getHighValue();
    - aggregateData.setHighValue(highValue);
    - aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    - if (ndvEstimator == null || !newData.isSetBitVectors() || newData.getBitVectors().length() == 0) {
    - aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + public ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException {
    + ColumnStatisticsObj statsObj = null;
    +
    + // check if all the ColumnStatisticsObjs contain stats and all the ndv are
    + // bitvectors
    + boolean doAllPartitionContainStats = partNames.size() == css.size();
    + boolean isNDVBitVectorSet = true;
    + String colType = null;
    + for (ColumnStatistics cs : css) {
    + if (cs.getStatsObjSize() != 1) {
    + throw new MetaException(
    + "The number of columns should be exactly one in aggrStats, but found "
    + + cs.getStatsObjSize());
    + }
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + if (statsObj == null) {
    + colType = cso.getColType();
    + statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType, cso
    + .getStatsData().getSetField());
    + }
    + if (numBitVectors <= 0 || !cso.getStatsData().getDecimalStats().isSetBitVectors()
    + || cso.getStatsData().getDecimalStats().getBitVectors().length() == 0) {
    + isNDVBitVectorSet = false;
    + break;
    + }
    + }
    + ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
    + if (doAllPartitionContainStats || css.size() < 2) {
    + DecimalColumnStatsData aggregateData = null;
    + long lowerBound = 0;
    + long higherBound = 0;
    + double densityAvgSum = 0.0;
    + NumDistinctValueEstimator ndvEstimator = null;
    + if (isNDVBitVectorSet) {
    + ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + }
    + for (ColumnStatistics cs : css) {
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + DecimalColumnStatsData newData = cso.getStatsData().getDecimalStats();
    + if (useDensityFunctionForNDVEstimation) {
    + lowerBound = Math.max(lowerBound, newData.getNumDVs());
    + higherBound += newData.getNumDVs();
    + densityAvgSum += (HBaseUtils.getDoubleValue(newData.getHighValue()) - HBaseUtils
    + .getDoubleValue(newData.getLowValue())) / newData.getNumDVs();
    + }
    + if (isNDVBitVectorSet) {
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + }
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + if (HBaseUtils.getDoubleValue(aggregateData.getLowValue()) < HBaseUtils
    + .getDoubleValue(newData.getLowValue())) {
    + aggregateData.setLowValue(aggregateData.getLowValue());
    + } else {
    + aggregateData.setLowValue(newData.getLowValue());
    + }
    + if (HBaseUtils.getDoubleValue(aggregateData.getHighValue()) > HBaseUtils
    + .getDoubleValue(newData.getHighValue())) {
    + aggregateData.setHighValue(aggregateData.getHighValue());
    + } else {
    + aggregateData.setHighValue(newData.getHighValue());
    + }
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + }
    + }
    + if (isNDVBitVectorSet) {
    + // if all the ColumnStatisticsObjs contain bitvectors, we do not need to
    + // use uniform distribution assumption because we can merge bitvectors
    + // to get a good estimation.
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + } else {
    + if (useDensityFunctionForNDVEstimation) {
    + // We have estimation, lowerbound and higherbound. We use estimation
    + // if it is between lowerbound and higherbound.
    + double densityAvg = densityAvgSum / partNames.size();
    + long estimation = (long) ((HBaseUtils.getDoubleValue(aggregateData.getHighValue()) - HBaseUtils
    + .getDoubleValue(aggregateData.getLowValue())) / densityAvg);
    + if (estimation < lowerBound) {
    + aggregateData.setNumDVs(lowerBound);
    + } else if (estimation > higherBound) {
    + aggregateData.setNumDVs(higherBound);
    + } else {
    + aggregateData.setNumDVs(estimation);
    + }
    + } else {
    + // Without useDensityFunctionForNDVEstimation, we just use the
    + // default one, which is the max of all the partitions and it is
    + // already done.
    + }
    + }
    + columnStatisticsData.setDecimalStats(aggregateData);
    + } else {
    + // we need extrapolation
    + Map<String, Integer> indexMap = new HashMap<String, Integer>();
    + for (int index = 0; index < partNames.size(); index++) {
    + indexMap.put(partNames.get(index), index);
    + }
    + Map<String, Double> adjustedIndexMap = new HashMap<String, Double>();
    + Map<String, ColumnStatisticsData> adjustedStatsMap = new HashMap<String, ColumnStatisticsData>();
    + // while we scan the css, we also get the densityAvg, lowerbound and
    + // higerbound when useDensityFunctionForNDVEstimation is true.
    + double densityAvgSum = 0.0;
    + if (!isNDVBitVectorSet) {
    + // if not every partition uses bitvector for ndv, we just fall back to
    + // the traditional extrapolation methods.
    + for (ColumnStatistics cs : css) {
    + String partName = cs.getStatsDesc().getPartName();
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + DecimalColumnStatsData newData = cso.getStatsData().getDecimalStats();
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (HBaseUtils.getDoubleValue(newData.getHighValue()) - HBaseUtils
    + .getDoubleValue(newData.getLowValue())) / newData.getNumDVs();
    + }
    + adjustedIndexMap.put(partName, (double) indexMap.get(partName));
    + adjustedStatsMap.put(partName, cso.getStatsData());
    + }
    + } else {
    + // we first merge all the adjacent bitvectors that we could merge and
    + // derive new partition names and index.
    + NumDistinctValueEstimator ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + StringBuilder pseudoPartName = new StringBuilder();
    + double pseudoIndexSum = 0;
    + int length = 0;
    + int curIndex = -1;
    + DecimalColumnStatsData aggregateData = null;
    + for (ColumnStatistics cs : css) {
    + String partName = cs.getStatsDesc().getPartName();
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + DecimalColumnStatsData newData = cso.getStatsData().getDecimalStats();
    + // newData.isSetBitVectors() should be true for sure because we
    + // already checked it before.
    + if (indexMap.get(partName) != curIndex) {
    + // There is bitvector, but it is not adjacent to the previous ones.
    + if (length > 0) {
    + // we have to set ndv
    + adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + ColumnStatisticsData csd = new ColumnStatisticsData();
    + csd.setDecimalStats(aggregateData);
    + adjustedStatsMap.put(pseudoPartName.toString(), csd);
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (HBaseUtils.getDoubleValue(aggregateData.getHighValue()) - HBaseUtils
    + .getDoubleValue(aggregateData.getLowValue())) / aggregateData.getNumDVs();
    + }
    + // reset everything
    + pseudoPartName = new StringBuilder();
    + pseudoIndexSum = 0;
    + length = 0;
    + }
    + aggregateData = null;
    + }
    + curIndex = indexMap.get(partName);
    + pseudoPartName.append(partName);
    + pseudoIndexSum += curIndex;
    + length++;
    + curIndex++;
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + if (HBaseUtils.getDoubleValue(aggregateData.getLowValue()) < HBaseUtils
    + .getDoubleValue(newData.getLowValue())) {
    + aggregateData.setLowValue(aggregateData.getLowValue());
    + } else {
    + aggregateData.setLowValue(newData.getLowValue());
    + }
    + if (HBaseUtils.getDoubleValue(aggregateData.getHighValue()) > HBaseUtils
    + .getDoubleValue(newData.getHighValue())) {
    + aggregateData.setHighValue(aggregateData.getHighValue());
    + } else {
    + aggregateData.setHighValue(newData.getHighValue());
    + }
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + }
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + }
    + if (length > 0) {
    + // we have to set ndv
    + adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + ColumnStatisticsData csd = new ColumnStatisticsData();
    + csd.setDecimalStats(aggregateData);
    + adjustedStatsMap.put(pseudoPartName.toString(), csd);
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (HBaseUtils.getDoubleValue(aggregateData.getHighValue()) - HBaseUtils
    + .getDoubleValue(aggregateData.getLowValue())) / aggregateData.getNumDVs();
    + }
    + }
    + }
    + extrapolate(columnStatisticsData, partNames.size(), css.size(), adjustedIndexMap,
    + adjustedStatsMap, densityAvgSum / adjustedStatsMap.size());
    + }
    + statsObj.setStatsData(columnStatisticsData);
    + return statsObj;
    + }
    +
    + @Override
    + public void extrapolate(ColumnStatisticsData extrapolateData, int numParts,
    + int numPartsWithStats, Map<String, Double> adjustedIndexMap,
    + Map<String, ColumnStatisticsData> adjustedStatsMap, double densityAvg) {
    + int rightBorderInd = numParts;
    + DecimalColumnStatsData extrapolateDecimalData = new DecimalColumnStatsData();
    + Map<String, DecimalColumnStatsData> extractedAdjustedStatsMap = new HashMap<>();
    + for (Map.Entry<String, ColumnStatisticsData> entry : adjustedStatsMap.entrySet()) {
    + extractedAdjustedStatsMap.put(entry.getKey(), entry.getValue().getDecimalStats());
    + }
    + List<Map.Entry<String, DecimalColumnStatsData>> list = new LinkedList<Map.Entry<String, DecimalColumnStatsData>>(
    + extractedAdjustedStatsMap.entrySet());
    + // get the lowValue
    + Collections.sort(list, new Comparator<Map.Entry<String, DecimalColumnStatsData>>() {
    + public int compare(Map.Entry<String, DecimalColumnStatsData> o1,
    + Map.Entry<String, DecimalColumnStatsData> o2) {
    + return o1.getValue().getLowValue().compareTo(o2.getValue().getLowValue());
    + }
    + });
    + double minInd = adjustedIndexMap.get(list.get(0).getKey());
    + double maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + double lowValue = 0;
    + double min = HBaseUtils.getDoubleValue(list.get(0).getValue().getLowValue());
    + double max = HBaseUtils.getDoubleValue(list.get(list.size() - 1).getValue().getLowValue());
    + if (minInd == maxInd) {
    + lowValue = min;
    + } else if (minInd < maxInd) {
    + // left border is the min
    + lowValue = (max - (max - min) * maxInd / (maxInd - minInd));
    + } else {
    + // right border is the min
    + lowValue = (max - (max - min) * (rightBorderInd - maxInd) / (minInd - maxInd));
    + }
    +
    + // get the highValue
    + Collections.sort(list, new Comparator<Map.Entry<String, DecimalColumnStatsData>>() {
    + public int compare(Map.Entry<String, DecimalColumnStatsData> o1,
    + Map.Entry<String, DecimalColumnStatsData> o2) {
    + return o1.getValue().getHighValue().compareTo(o2.getValue().getHighValue());
    + }
    + });
    + minInd = adjustedIndexMap.get(list.get(0).getKey());
    + maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + double highValue = 0;
    + min = HBaseUtils.getDoubleValue(list.get(0).getValue().getHighValue());
    + max = HBaseUtils.getDoubleValue(list.get(list.size() - 1).getValue().getHighValue());
    + if (minInd == maxInd) {
    + highValue = min;
    + } else if (minInd < maxInd) {
    + // right border is the max
    + highValue = (min + (max - min) * (rightBorderInd - minInd) / (maxInd - minInd));
    + } else {
    + // left border is the max
    + highValue = (min + (max - min) * minInd / (minInd - maxInd));
    + }
    +
    + // get the #nulls
    + long numNulls = 0;
    + for (Map.Entry<String, DecimalColumnStatsData> entry : extractedAdjustedStatsMap.entrySet()) {
    + numNulls += entry.getValue().getNumNulls();
    + }
    + // we scale up sumNulls based on the number of partitions
    + numNulls = numNulls * numParts / numPartsWithStats;
    +
    + // get the ndv
    + long ndv = 0;
    + long ndvMin = 0;
    + long ndvMax = 0;
    + Collections.sort(list, new Comparator<Map.Entry<String, DecimalColumnStatsData>>() {
    + public int compare(Map.Entry<String, DecimalColumnStatsData> o1,
    + Map.Entry<String, DecimalColumnStatsData> o2) {
    + return o1.getValue().getNumDVs() < o2.getValue().getNumDVs() ? -1 : 1;
    + }
    + });
    + long lowerBound = list.get(list.size() - 1).getValue().getNumDVs();
    + long higherBound = 0;
    + for (Map.Entry<String, DecimalColumnStatsData> entry : list) {
    + higherBound += entry.getValue().getNumDVs();
    + }
    + if (useDensityFunctionForNDVEstimation && densityAvg != 0.0) {
    + ndv = (long) ((highValue - lowValue) / densityAvg);
    + if (ndv < lowerBound) {
    + ndv = lowerBound;
    + } else if (ndv > higherBound) {
    + ndv = higherBound;
    + }
          } else {
    - ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    - ndvEstimator.getnumBitVectors()));
    - aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    - aggregateData.setBitVectors(ndvEstimator.serialize().toString());
    + minInd = adjustedIndexMap.get(list.get(0).getKey());
    + maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + ndvMin = list.get(0).getValue().getNumDVs();
    + ndvMax = list.get(list.size() - 1).getValue().getNumDVs();
    + if (minInd == maxInd) {
    + ndv = ndvMin;
    + } else if (minInd < maxInd) {
    + // right border is the max
    + ndv = (long) (ndvMin + (ndvMax - ndvMin) * (rightBorderInd - minInd) / (maxInd - minInd));
    + } else {
    + // left border is the max
    + ndv = (long) (ndvMin + (ndvMax - ndvMin) * minInd / (minInd - maxInd));
    + }
          }
    + extrapolateDecimalData.setLowValue(StatObjectConverter.createThriftDecimal(String
    + .valueOf(lowValue)));
    + extrapolateDecimalData.setHighValue(StatObjectConverter.createThriftDecimal(String
    + .valueOf(highValue)));
    + extrapolateDecimalData.setNumNulls(numNulls);
    + extrapolateDecimalData.setNumDVs(ndv);
    + extrapolateData.setDecimalStats(extrapolateDecimalData);
        }
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DoubleColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DoubleColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DoubleColumnStatsAggregator.java
    index d945ec2..a88ef84 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DoubleColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/DoubleColumnStatsAggregator.java
    @@ -19,26 +19,307 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
      import org.apache.hadoop.hive.metastore.NumDistinctValueEstimator;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
      import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.MetaException;

    -public class DoubleColumnStatsAggregator extends ColumnStatsAggregator {
    +public class DoubleColumnStatsAggregator extends ColumnStatsAggregator implements
    + IExtrapolatePartStatus {

        @Override
    - public void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats) {
    - DoubleColumnStatsData aggregateData = aggregateColStats.getStatsData().getDoubleStats();
    - DoubleColumnStatsData newData = newColStats.getStatsData().getDoubleStats();
    - aggregateData.setLowValue(Math.min(aggregateData.getLowValue(), newData.getLowValue()));
    - aggregateData.setHighValue(Math.max(aggregateData.getHighValue(), newData.getHighValue()));
    - aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    - if (ndvEstimator == null || !newData.isSetBitVectors() || newData.getBitVectors().length() == 0) {
    - aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + public ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException {
    + ColumnStatisticsObj statsObj = null;
    +
    + // check if all the ColumnStatisticsObjs contain stats and all the ndv are
    + // bitvectors
    + boolean doAllPartitionContainStats = partNames.size() == css.size();
    + boolean isNDVBitVectorSet = true;
    + String colType = null;
    + for (ColumnStatistics cs : css) {
    + if (cs.getStatsObjSize() != 1) {
    + throw new MetaException(
    + "The number of columns should be exactly one in aggrStats, but found "
    + + cs.getStatsObjSize());
    + }
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + if (statsObj == null) {
    + colType = cso.getColType();
    + statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType, cso
    + .getStatsData().getSetField());
    + }
    + if (numBitVectors <= 0 || !cso.getStatsData().getDoubleStats().isSetBitVectors()
    + || cso.getStatsData().getDoubleStats().getBitVectors().length() == 0) {
    + isNDVBitVectorSet = false;
    + break;
    + }
    + }
    + ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
    + if (doAllPartitionContainStats || css.size() < 2) {
    + DoubleColumnStatsData aggregateData = null;
    + long lowerBound = 0;
    + long higherBound = 0;
    + double densityAvgSum = 0.0;
    + NumDistinctValueEstimator ndvEstimator = null;
    + if (isNDVBitVectorSet) {
    + ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + }
    + for (ColumnStatistics cs : css) {
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + DoubleColumnStatsData newData = cso.getStatsData().getDoubleStats();
    + if (useDensityFunctionForNDVEstimation) {
    + lowerBound = Math.max(lowerBound, newData.getNumDVs());
    + higherBound += newData.getNumDVs();
    + densityAvgSum += (newData.getHighValue() - newData.getLowValue()) / newData.getNumDVs();
    + }
    + if (isNDVBitVectorSet) {
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + }
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData.setLowValue(Math.min(aggregateData.getLowValue(), newData.getLowValue()));
    + aggregateData
    + .setHighValue(Math.max(aggregateData.getHighValue(), newData.getHighValue()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + }
    + }
    + if (isNDVBitVectorSet) {
    + // if all the ColumnStatisticsObjs contain bitvectors, we do not need to
    + // use uniform distribution assumption because we can merge bitvectors
    + // to get a good estimation.
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + } else {
    + if (useDensityFunctionForNDVEstimation) {
    + // We have estimation, lowerbound and higherbound. We use estimation
    + // if it is between lowerbound and higherbound.
    + double densityAvg = densityAvgSum / partNames.size();
    + long estimation = (long) ((aggregateData.getHighValue() - aggregateData.getLowValue()) / densityAvg);
    + if (estimation < lowerBound) {
    + aggregateData.setNumDVs(lowerBound);
    + } else if (estimation > higherBound) {
    + aggregateData.setNumDVs(higherBound);
    + } else {
    + aggregateData.setNumDVs(estimation);
    + }
    + } else {
    + // Without useDensityFunctionForNDVEstimation, we just use the
    + // default one, which is the max of all the partitions and it is
    + // already done.
    + }
    + }
    + columnStatisticsData.setDoubleStats(aggregateData);
    + } else {
    + // we need extrapolation
    + Map<String, Integer> indexMap = new HashMap<String, Integer>();
    + for (int index = 0; index < partNames.size(); index++) {
    + indexMap.put(partNames.get(index), index);
    + }
    + Map<String, Double> adjustedIndexMap = new HashMap<String, Double>();
    + Map<String, ColumnStatisticsData> adjustedStatsMap = new HashMap<String, ColumnStatisticsData>();
    + // while we scan the css, we also get the densityAvg, lowerbound and
    + // higerbound when useDensityFunctionForNDVEstimation is true.
    + double densityAvgSum = 0.0;
    + if (!isNDVBitVectorSet) {
    + // if not every partition uses bitvector for ndv, we just fall back to
    + // the traditional extrapolation methods.
    + for (ColumnStatistics cs : css) {
    + String partName = cs.getStatsDesc().getPartName();
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + DoubleColumnStatsData newData = cso.getStatsData().getDoubleStats();
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (newData.getHighValue() - newData.getLowValue()) / newData.getNumDVs();
    + }
    + adjustedIndexMap.put(partName, (double) indexMap.get(partName));
    + adjustedStatsMap.put(partName, cso.getStatsData());
    + }
    + } else {
    + // we first merge all the adjacent bitvectors that we could merge and
    + // derive new partition names and index.
    + NumDistinctValueEstimator ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + StringBuilder pseudoPartName = new StringBuilder();
    + double pseudoIndexSum = 0;
    + int length = 0;
    + int curIndex = -1;
    + DoubleColumnStatsData aggregateData = null;
    + for (ColumnStatistics cs : css) {
    + String partName = cs.getStatsDesc().getPartName();
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + DoubleColumnStatsData newData = cso.getStatsData().getDoubleStats();
    + // newData.isSetBitVectors() should be true for sure because we
    + // already checked it before.
    + if (indexMap.get(partName) != curIndex) {
    + // There is bitvector, but it is not adjacent to the previous ones.
    + if (length > 0) {
    + // we have to set ndv
    + adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + ColumnStatisticsData csd = new ColumnStatisticsData();
    + csd.setDoubleStats(aggregateData);
    + adjustedStatsMap.put(pseudoPartName.toString(), csd);
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (aggregateData.getHighValue() - aggregateData.getLowValue()) / aggregateData.getNumDVs();
    + }
    + // reset everything
    + pseudoPartName = new StringBuilder();
    + pseudoIndexSum = 0;
    + length = 0;
    + }
    + aggregateData = null;
    + }
    + curIndex = indexMap.get(partName);
    + pseudoPartName.append(partName);
    + pseudoIndexSum += curIndex;
    + length++;
    + curIndex++;
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData.setLowValue(Math.min(aggregateData.getLowValue(), newData.getLowValue()));
    + aggregateData.setHighValue(Math.max(aggregateData.getHighValue(),
    + newData.getHighValue()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + }
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + }
    + if (length > 0) {
    + // we have to set ndv
    + adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + ColumnStatisticsData csd = new ColumnStatisticsData();
    + csd.setDoubleStats(aggregateData);
    + adjustedStatsMap.put(pseudoPartName.toString(), csd);
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (aggregateData.getHighValue() - aggregateData.getLowValue()) / aggregateData.getNumDVs();
    + }
    + }
    + }
    + extrapolate(columnStatisticsData, partNames.size(), css.size(), adjustedIndexMap,
    + adjustedStatsMap, densityAvgSum / adjustedStatsMap.size());
    + }
    + statsObj.setStatsData(columnStatisticsData);
    + return statsObj;
    + }
    +
    + @Override
    + public void extrapolate(ColumnStatisticsData extrapolateData, int numParts,
    + int numPartsWithStats, Map<String, Double> adjustedIndexMap,
    + Map<String, ColumnStatisticsData> adjustedStatsMap, double densityAvg) {
    + int rightBorderInd = numParts;
    + DoubleColumnStatsData extrapolateDoubleData = new DoubleColumnStatsData();
    + Map<String, DoubleColumnStatsData> extractedAdjustedStatsMap = new HashMap<>();
    + for (Map.Entry<String, ColumnStatisticsData> entry : adjustedStatsMap.entrySet()) {
    + extractedAdjustedStatsMap.put(entry.getKey(), entry.getValue().getDoubleStats());
    + }
    + List<Map.Entry<String, DoubleColumnStatsData>> list = new LinkedList<Map.Entry<String, DoubleColumnStatsData>>(
    + extractedAdjustedStatsMap.entrySet());
    + // get the lowValue
    + Collections.sort(list, new Comparator<Map.Entry<String, DoubleColumnStatsData>>() {
    + public int compare(Map.Entry<String, DoubleColumnStatsData> o1,
    + Map.Entry<String, DoubleColumnStatsData> o2) {
    + return o1.getValue().getLowValue() < o2.getValue().getLowValue() ? -1 : 1;
    + }
    + });
    + double minInd = adjustedIndexMap.get(list.get(0).getKey());
    + double maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + double lowValue = 0;
    + double min = list.get(0).getValue().getLowValue();
    + double max = list.get(list.size() - 1).getValue().getLowValue();
    + if (minInd == maxInd) {
    + lowValue = min;
    + } else if (minInd < maxInd) {
    + // left border is the min
    + lowValue = (max - (max - min) * maxInd / (maxInd - minInd));
    + } else {
    + // right border is the min
    + lowValue = (max - (max - min) * (rightBorderInd - maxInd) / (minInd - maxInd));
    + }
    +
    + // get the highValue
    + Collections.sort(list, new Comparator<Map.Entry<String, DoubleColumnStatsData>>() {
    + public int compare(Map.Entry<String, DoubleColumnStatsData> o1,
    + Map.Entry<String, DoubleColumnStatsData> o2) {
    + return o1.getValue().getHighValue() < o2.getValue().getHighValue() ? -1 : 1;
    + }
    + });
    + minInd = adjustedIndexMap.get(list.get(0).getKey());
    + maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + double highValue = 0;
    + min = list.get(0).getValue().getHighValue();
    + max = list.get(list.size() - 1).getValue().getHighValue();
    + if (minInd == maxInd) {
    + highValue = min;
    + } else if (minInd < maxInd) {
    + // right border is the max
    + highValue = (min + (max - min) * (rightBorderInd - minInd) / (maxInd - minInd));
    + } else {
    + // left border is the max
    + highValue = (min + (max - min) * minInd / (minInd - maxInd));
    + }
    +
    + // get the #nulls
    + long numNulls = 0;
    + for (Map.Entry<String, DoubleColumnStatsData> entry : extractedAdjustedStatsMap.entrySet()) {
    + numNulls += entry.getValue().getNumNulls();
    + }
    + // we scale up sumNulls based on the number of partitions
    + numNulls = numNulls * numParts / numPartsWithStats;
    +
    + // get the ndv
    + long ndv = 0;
    + long ndvMin = 0;
    + long ndvMax = 0;
    + Collections.sort(list, new Comparator<Map.Entry<String, DoubleColumnStatsData>>() {
    + public int compare(Map.Entry<String, DoubleColumnStatsData> o1,
    + Map.Entry<String, DoubleColumnStatsData> o2) {
    + return o1.getValue().getNumDVs() < o2.getValue().getNumDVs() ? -1 : 1;
    + }
    + });
    + long lowerBound = list.get(list.size() - 1).getValue().getNumDVs();
    + long higherBound = 0;
    + for (Map.Entry<String, DoubleColumnStatsData> entry : list) {
    + higherBound += entry.getValue().getNumDVs();
    + }
    + if (useDensityFunctionForNDVEstimation && densityAvg != 0.0) {
    + ndv = (long) ((highValue - lowValue) / densityAvg);
    + if (ndv < lowerBound) {
    + ndv = lowerBound;
    + } else if (ndv > higherBound) {
    + ndv = higherBound;
    + }
          } else {
    - ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    - ndvEstimator.getnumBitVectors()));
    - aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    - aggregateData.setBitVectors(ndvEstimator.serialize().toString());
    + minInd = adjustedIndexMap.get(list.get(0).getKey());
    + maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + ndvMin = list.get(0).getValue().getNumDVs();
    + ndvMax = list.get(list.size() - 1).getValue().getNumDVs();
    + if (minInd == maxInd) {
    + ndv = ndvMin;
    + } else if (minInd < maxInd) {
    + // right border is the max
    + ndv = (long) (ndvMin + (ndvMax - ndvMin) * (rightBorderInd - minInd) / (maxInd - minInd));
    + } else {
    + // left border is the max
    + ndv = (long) (ndvMin + (ndvMax - ndvMin) * minInd / (minInd - maxInd));
    + }
          }
    + extrapolateDoubleData.setLowValue(lowValue);
    + extrapolateDoubleData.setHighValue(highValue);
    + extrapolateDoubleData.setNumNulls(numNulls);
    + extrapolateDoubleData.setNumDVs(ndv);
    + extrapolateData.setDoubleStats(extrapolateDoubleData);
        }
    +
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/IExtrapolatePartStatus.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/IExtrapolatePartStatus.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/IExtrapolatePartStatus.java
    new file mode 100644
    index 0000000..99af060
    --- /dev/null
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/IExtrapolatePartStatus.java
    @@ -0,0 +1,30 @@
    +package org.apache.hadoop.hive.metastore.hbase.stats;
    +
    +import java.util.Map;
    +
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
    +
    +public interface IExtrapolatePartStatus {
    + // The following function will extrapolate the stats when the column stats of
    + // some partitions are missing.
    + /**
    + * @param extrapolateData
    + * it will carry back the specific stats, e.g., DOUBLE_STATS or
    + * LONG_STATS
    + * @param numParts
    + * the total number of partitions
    + * @param numPartsWithStats
    + * the number of partitions that have stats
    + * @param adjustedIndexMap
    + * the partition name to index map
    + * @param adjustedStatsMap
    + * the partition name to its stats map
    + * @param densityAvg
    + * the average of ndv density, which is useful when
    + * useDensityFunctionForNDVEstimation is true.
    + */
    + public abstract void extrapolate(ColumnStatisticsData extrapolateData, int numParts,
    + int numPartsWithStats, Map<String, Double> adjustedIndexMap,
    + Map<String, ColumnStatisticsData> adjustedStatsMap, double densityAvg);
    +
    +}

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/LongColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/LongColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/LongColumnStatsAggregator.java
    index 068dd00..8ac6561 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/LongColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/LongColumnStatsAggregator.java
    @@ -19,26 +19,305 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    +import java.util.Collections;
    +import java.util.Comparator;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +
      import org.apache.hadoop.hive.metastore.NumDistinctValueEstimator;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
      import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.MetaException;

    -public class LongColumnStatsAggregator extends ColumnStatsAggregator {
    +public class LongColumnStatsAggregator extends ColumnStatsAggregator implements
    + IExtrapolatePartStatus {

        @Override
    - public void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats) {
    - LongColumnStatsData aggregateData = aggregateColStats.getStatsData().getLongStats();
    - LongColumnStatsData newData = newColStats.getStatsData().getLongStats();
    - aggregateData.setLowValue(Math.min(aggregateData.getLowValue(), newData.getLowValue()));
    - aggregateData.setHighValue(Math.max(aggregateData.getHighValue(), newData.getHighValue()));
    - aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    - if (ndvEstimator == null || !newData.isSetBitVectors() || newData.getBitVectors().length() == 0) {
    - aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + public ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException {
    + ColumnStatisticsObj statsObj = null;
    +
    + // check if all the ColumnStatisticsObjs contain stats and all the ndv are
    + // bitvectors
    + boolean doAllPartitionContainStats = partNames.size() == css.size();
    + boolean isNDVBitVectorSet = true;
    + String colType = null;
    + for (ColumnStatistics cs : css) {
    + if (cs.getStatsObjSize() != 1) {
    + throw new MetaException(
    + "The number of columns should be exactly one in aggrStats, but found "
    + + cs.getStatsObjSize());
    + }
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + if (statsObj == null) {
    + colType = cso.getColType();
    + statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType, cso
    + .getStatsData().getSetField());
    + }
    + if (numBitVectors <= 0 || !cso.getStatsData().getLongStats().isSetBitVectors()
    + || cso.getStatsData().getLongStats().getBitVectors().length() == 0) {
    + isNDVBitVectorSet = false;
    + break;
    + }
    + }
    + ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
    + if (doAllPartitionContainStats || css.size() < 2) {
    + LongColumnStatsData aggregateData = null;
    + long lowerBound = 0;
    + long higherBound = 0;
    + double densityAvgSum = 0.0;
    + NumDistinctValueEstimator ndvEstimator = null;
    + if (isNDVBitVectorSet) {
    + ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + }
    + for (ColumnStatistics cs : css) {
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + LongColumnStatsData newData = cso.getStatsData().getLongStats();
    + if (useDensityFunctionForNDVEstimation) {
    + lowerBound = Math.max(lowerBound, newData.getNumDVs());
    + higherBound += newData.getNumDVs();
    + densityAvgSum += (newData.getHighValue() - newData.getLowValue()) / newData.getNumDVs();
    + }
    + if (isNDVBitVectorSet) {
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + }
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData.setLowValue(Math.min(aggregateData.getLowValue(), newData.getLowValue()));
    + aggregateData
    + .setHighValue(Math.max(aggregateData.getHighValue(), newData.getHighValue()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + }
    + }
    + if (isNDVBitVectorSet) {
    + // if all the ColumnStatisticsObjs contain bitvectors, we do not need to
    + // use uniform distribution assumption because we can merge bitvectors
    + // to get a good estimation.
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + } else {
    + if (useDensityFunctionForNDVEstimation) {
    + // We have estimation, lowerbound and higherbound. We use estimation
    + // if it is between lowerbound and higherbound.
    + double densityAvg = densityAvgSum / partNames.size();
    + long estimation = (long) ((aggregateData.getHighValue() - aggregateData.getLowValue()) / densityAvg);
    + if (estimation < lowerBound) {
    + aggregateData.setNumDVs(lowerBound);
    + } else if (estimation > higherBound) {
    + aggregateData.setNumDVs(higherBound);
    + } else {
    + aggregateData.setNumDVs(estimation);
    + }
    + } else {
    + // Without useDensityFunctionForNDVEstimation, we just use the
    + // default one, which is the max of all the partitions and it is
    + // already done.
    + }
    + }
    + columnStatisticsData.setLongStats(aggregateData);
          } else {
    - ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    - ndvEstimator.getnumBitVectors()));
    - aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    - aggregateData.setBitVectors(ndvEstimator.serialize().toString());
    + // we need extrapolation
    + Map<String, Integer> indexMap = new HashMap<String, Integer>();
    + for (int index = 0; index < partNames.size(); index++) {
    + indexMap.put(partNames.get(index), index);
    + }
    + Map<String, Double> adjustedIndexMap = new HashMap<String, Double>();
    + Map<String, ColumnStatisticsData> adjustedStatsMap = new HashMap<String, ColumnStatisticsData>();
    + // while we scan the css, we also get the densityAvg, lowerbound and
    + // higerbound when useDensityFunctionForNDVEstimation is true.
    + double densityAvgSum = 0.0;
    + if (!isNDVBitVectorSet) {
    + // if not every partition uses bitvector for ndv, we just fall back to
    + // the traditional extrapolation methods.
    + for (ColumnStatistics cs : css) {
    + String partName = cs.getStatsDesc().getPartName();
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + LongColumnStatsData newData = cso.getStatsData().getLongStats();
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (newData.getHighValue() - newData.getLowValue()) / newData.getNumDVs();
    + }
    + adjustedIndexMap.put(partName, (double) indexMap.get(partName));
    + adjustedStatsMap.put(partName, cso.getStatsData());
    + }
    + } else {
    + // we first merge all the adjacent bitvectors that we could merge and
    + // derive new partition names and index.
    + NumDistinctValueEstimator ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + StringBuilder pseudoPartName = new StringBuilder();
    + double pseudoIndexSum = 0;
    + int length = 0;
    + int curIndex = -1;
    + LongColumnStatsData aggregateData = null;
    + for (ColumnStatistics cs : css) {
    + String partName = cs.getStatsDesc().getPartName();
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + LongColumnStatsData newData = cso.getStatsData().getLongStats();
    + // newData.isSetBitVectors() should be true for sure because we
    + // already checked it before.
    + if (indexMap.get(partName) != curIndex) {
    + // There is bitvector, but it is not adjacent to the previous ones.
    + if (length > 0) {
    + // we have to set ndv
    + adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + ColumnStatisticsData csd = new ColumnStatisticsData();
    + csd.setLongStats(aggregateData);
    + adjustedStatsMap.put(pseudoPartName.toString(), csd);
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (aggregateData.getHighValue() - aggregateData.getLowValue()) / aggregateData.getNumDVs();
    + }
    + // reset everything
    + pseudoPartName = new StringBuilder();
    + pseudoIndexSum = 0;
    + length = 0;
    + }
    + aggregateData = null;
    + }
    + curIndex = indexMap.get(partName);
    + pseudoPartName.append(partName);
    + pseudoIndexSum += curIndex;
    + length++;
    + curIndex++;
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData.setLowValue(Math.min(aggregateData.getLowValue(), newData.getLowValue()));
    + aggregateData.setHighValue(Math.max(aggregateData.getHighValue(),
    + newData.getHighValue()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + }
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + }
    + if (length > 0) {
    + // we have to set ndv
    + adjustedIndexMap.put(pseudoPartName.toString(), pseudoIndexSum / length);
    + aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    + ColumnStatisticsData csd = new ColumnStatisticsData();
    + csd.setLongStats(aggregateData);
    + adjustedStatsMap.put(pseudoPartName.toString(), csd);
    + if (useDensityFunctionForNDVEstimation) {
    + densityAvgSum += (aggregateData.getHighValue() - aggregateData.getLowValue()) / aggregateData.getNumDVs();
    + }
    + }
    + }
    + extrapolate(columnStatisticsData, partNames.size(), css.size(), adjustedIndexMap,
    + adjustedStatsMap, densityAvgSum / adjustedStatsMap.size());
          }
    + statsObj.setStatsData(columnStatisticsData);
    + return statsObj;
        }
    +
    + @Override
    + public void extrapolate(ColumnStatisticsData extrapolateData, int numParts,
    + int numPartsWithStats, Map<String, Double> adjustedIndexMap,
    + Map<String, ColumnStatisticsData> adjustedStatsMap, double densityAvg) {
    + int rightBorderInd = numParts;
    + LongColumnStatsData extrapolateLongData = new LongColumnStatsData();
    + Map<String, LongColumnStatsData> extractedAdjustedStatsMap = new HashMap<>();
    + for (Map.Entry<String, ColumnStatisticsData> entry : adjustedStatsMap.entrySet()) {
    + extractedAdjustedStatsMap.put(entry.getKey(), entry.getValue().getLongStats());
    + }
    + List<Map.Entry<String, LongColumnStatsData>> list = new LinkedList<Map.Entry<String, LongColumnStatsData>>(
    + extractedAdjustedStatsMap.entrySet());
    + // get the lowValue
    + Collections.sort(list, new Comparator<Map.Entry<String, LongColumnStatsData>>() {
    + public int compare(Map.Entry<String, LongColumnStatsData> o1,
    + Map.Entry<String, LongColumnStatsData> o2) {
    + return o1.getValue().getLowValue() < o2.getValue().getLowValue() ? -1 : 1;
    + }
    + });
    + double minInd = adjustedIndexMap.get(list.get(0).getKey());
    + double maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + long lowValue = 0;
    + long min = list.get(0).getValue().getLowValue();
    + long max = list.get(list.size() - 1).getValue().getLowValue();
    + if (minInd == maxInd) {
    + lowValue = min;
    + } else if (minInd < maxInd) {
    + // left border is the min
    + lowValue = (long) (max - (max - min) * maxInd / (maxInd - minInd));
    + } else {
    + // right border is the min
    + lowValue = (long) (max - (max - min) * (rightBorderInd - maxInd) / (minInd - maxInd));
    + }
    +
    + // get the highValue
    + Collections.sort(list, new Comparator<Map.Entry<String, LongColumnStatsData>>() {
    + public int compare(Map.Entry<String, LongColumnStatsData> o1,
    + Map.Entry<String, LongColumnStatsData> o2) {
    + return o1.getValue().getHighValue() < o2.getValue().getHighValue() ? -1 : 1;
    + }
    + });
    + minInd = adjustedIndexMap.get(list.get(0).getKey());
    + maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + long highValue = 0;
    + min = list.get(0).getValue().getHighValue();
    + max = list.get(list.size() - 1).getValue().getHighValue();
    + if (minInd == maxInd) {
    + highValue = min;
    + } else if (minInd < maxInd) {
    + // right border is the max
    + highValue = (long) (min + (max - min) * (rightBorderInd - minInd) / (maxInd - minInd));
    + } else {
    + // left border is the max
    + highValue = (long) (min + (max - min) * minInd / (minInd - maxInd));
    + }
    +
    + // get the #nulls
    + long numNulls = 0;
    + for (Map.Entry<String, LongColumnStatsData> entry : extractedAdjustedStatsMap.entrySet()) {
    + numNulls += entry.getValue().getNumNulls();
    + }
    + // we scale up sumNulls based on the number of partitions
    + numNulls = numNulls * numParts / numPartsWithStats;
    +
    + // get the ndv
    + long ndv = 0;
    + Collections.sort(list, new Comparator<Map.Entry<String, LongColumnStatsData>>() {
    + public int compare(Map.Entry<String, LongColumnStatsData> o1,
    + Map.Entry<String, LongColumnStatsData> o2) {
    + return o1.getValue().getNumDVs() < o2.getValue().getNumDVs() ? -1 : 1;
    + }
    + });
    + long lowerBound = list.get(list.size() - 1).getValue().getNumDVs();
    + long higherBound = 0;
    + for (Map.Entry<String, LongColumnStatsData> entry : list) {
    + higherBound += entry.getValue().getNumDVs();
    + }
    + if (useDensityFunctionForNDVEstimation && densityAvg != 0.0) {
    + ndv = (long) ((highValue - lowValue) / densityAvg);
    + if (ndv < lowerBound) {
    + ndv = lowerBound;
    + } else if (ndv > higherBound) {
    + ndv = higherBound;
    + }
    + } else {
    + minInd = adjustedIndexMap.get(list.get(0).getKey());
    + maxInd = adjustedIndexMap.get(list.get(list.size() - 1).getKey());
    + min = list.get(0).getValue().getNumDVs();
    + max = list.get(list.size() - 1).getValue().getNumDVs();
    + if (minInd == maxInd) {
    + ndv = min;
    + } else if (minInd < maxInd) {
    + // right border is the max
    + ndv = (long) (min + (max - min) * (rightBorderInd - minInd) / (maxInd - minInd));
    + } else {
    + // left border is the max
    + ndv = (long) (min + (max - min) * minInd / (minInd - maxInd));
    + }
    + }
    + extrapolateLongData.setLowValue(lowValue);
    + extrapolateLongData.setHighValue(highValue);
    + extrapolateLongData.setNumNulls(numNulls);
    + extrapolateLongData.setNumDVs(ndv);
    + extrapolateData.setLongStats(extrapolateLongData);
    + }
    +
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/StringColumnStatsAggregator.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/StringColumnStatsAggregator.java b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/StringColumnStatsAggregator.java
    index aeb6c39..2aa4046 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/StringColumnStatsAggregator.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/hbase/stats/StringColumnStatsAggregator.java
    @@ -19,26 +19,87 @@

      package org.apache.hadoop.hive.metastore.hbase.stats;

    +import java.util.List;
    +
      import org.apache.hadoop.hive.metastore.NumDistinctValueEstimator;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
      import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    +import org.apache.hadoop.hive.metastore.api.MetaException;
      import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;

      public class StringColumnStatsAggregator extends ColumnStatsAggregator {

        @Override
    - public void aggregate(ColumnStatisticsObj aggregateColStats, ColumnStatisticsObj newColStats) {
    - StringColumnStatsData aggregateData = aggregateColStats.getStatsData().getStringStats();
    - StringColumnStatsData newData = newColStats.getStatsData().getStringStats();
    - aggregateData.setMaxColLen(Math.max(aggregateData.getMaxColLen(), newData.getMaxColLen()));
    - aggregateData.setAvgColLen(Math.max(aggregateData.getAvgColLen(), newData.getAvgColLen()));
    - aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    - if (ndvEstimator == null || !newData.isSetBitVectors() || newData.getBitVectors().length() == 0) {
    - aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    - } else {
    - ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    - ndvEstimator.getnumBitVectors()));
    + public ColumnStatisticsObj aggregate(String colName, List<String> partNames,
    + List<ColumnStatistics> css) throws MetaException {
    + ColumnStatisticsObj statsObj = null;
    +
    + // check if all the ColumnStatisticsObjs contain stats and all the ndv are
    + // bitvectors. Only when both of the conditions are true, we merge bit
    + // vectors. Otherwise, just use the maximum function.
    + boolean doAllPartitionContainStats = partNames.size() == css.size();
    + boolean isNDVBitVectorSet = true;
    + String colType = null;
    + for (ColumnStatistics cs : css) {
    + if (cs.getStatsObjSize() != 1) {
    + throw new MetaException(
    + "The number of columns should be exactly one in aggrStats, but found "
    + + cs.getStatsObjSize());
    + }
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + if (statsObj == null) {
    + colType = cso.getColType();
    + statsObj = ColumnStatsAggregatorFactory.newColumnStaticsObj(colName, colType, cso
    + .getStatsData().getSetField());
    + }
    + if (numBitVectors <= 0 || !cso.getStatsData().getStringStats().isSetBitVectors()
    + || cso.getStatsData().getStringStats().getBitVectors().length() == 0) {
    + isNDVBitVectorSet = false;
    + break;
    + }
    + }
    + ColumnStatisticsData columnStatisticsData = new ColumnStatisticsData();
    + if (doAllPartitionContainStats && isNDVBitVectorSet) {
    + StringColumnStatsData aggregateData = null;
    + NumDistinctValueEstimator ndvEstimator = new NumDistinctValueEstimator(numBitVectors);
    + for (ColumnStatistics cs : css) {
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + StringColumnStatsData newData = cso.getStatsData().getStringStats();
    + ndvEstimator.mergeEstimators(new NumDistinctValueEstimator(newData.getBitVectors(),
    + ndvEstimator.getnumBitVectors()));
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData
    + .setMaxColLen(Math.max(aggregateData.getMaxColLen(), newData.getMaxColLen()));
    + aggregateData
    + .setAvgColLen(Math.max(aggregateData.getAvgColLen(), newData.getAvgColLen()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + }
    + }
            aggregateData.setNumDVs(ndvEstimator.estimateNumDistinctValues());
    - aggregateData.setBitVectors(ndvEstimator.serialize().toString());
    + columnStatisticsData.setStringStats(aggregateData);
    + } else {
    + StringColumnStatsData aggregateData = null;
    + for (ColumnStatistics cs : css) {
    + ColumnStatisticsObj cso = cs.getStatsObjIterator().next();
    + StringColumnStatsData newData = cso.getStatsData().getStringStats();
    + if (aggregateData == null) {
    + aggregateData = newData.deepCopy();
    + } else {
    + aggregateData
    + .setMaxColLen(Math.max(aggregateData.getMaxColLen(), newData.getMaxColLen()));
    + aggregateData
    + .setAvgColLen(Math.max(aggregateData.getAvgColLen(), newData.getAvgColLen()));
    + aggregateData.setNumNulls(aggregateData.getNumNulls() + newData.getNumNulls());
    + aggregateData.setNumDVs(Math.max(aggregateData.getNumDVs(), newData.getNumDVs()));
    + }
    + }
    + columnStatisticsData.setStringStats(aggregateData);
          }
    + statsObj.setStatsData(columnStatisticsData);
    + return statsObj;
        }
    +
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
    index 36c7984..e0c4094 100644
    --- a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
    +++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsCacheWithBitVector.java
    @@ -156,10 +156,8 @@ public class TestHBaseAggregateStatsCacheWithBitVector {
              Assert.assertEquals(-20.12, dcsd.getLowValue(), 0.01);
              Assert.assertEquals(60, dcsd.getNumNulls());
              Assert.assertEquals(5, dcsd.getNumDVs());
    - Assert
    - .assertEquals(
    - "{0, 1, 4, 5, 7}{0, 1}{0, 1, 2, 4}{0, 1, 2, 4}{0, 1, 2}{0, 2}{0, 1, 3, 4}{0, 1, 2, 3, 4}{0, 1, 4}{0, 1, 3, 4, 6}{0, 2}{0, 1, 3, 8}{0, 2, 3}{0, 2}{0, 1, 9}{0, 1, 4}",
    - dcsd.getBitVectors());
    + // we do not store the bitvector for the aggrStats.
    + // we can store that if it is necessary in the future.
            }
          };
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13307: LLAP: Slider package should contain permanent functions (addendum)


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

    Branch: refs/heads/llap
    Commit: b1c45029ed3652eda9db6650da38ba653d4ada93
    Parents: 4fabd03
    Author: Gopal V <gopalv@apache.org>
    Authored: Fri Mar 25 00:18:44 2016 -0700
    Committer: Gopal V <gopalv@apache.org>
    Committed: Fri Mar 25 00:19:35 2016 -0700

    ----------------------------------------------------------------------
      ql/src/test/queries/clientpositive/llap_udf.q | 6 +++---
      1 file changed, 3 insertions(+), 3 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/b1c45029/ql/src/test/queries/clientpositive/llap_udf.q
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/queries/clientpositive/llap_udf.q b/ql/src/test/queries/clientpositive/llap_udf.q
    index c964f2b..2224bd5 100644
    --- a/ql/src/test/queries/clientpositive/llap_udf.q
    +++ b/ql/src/test/queries/clientpositive/llap_udf.q
    @@ -3,7 +3,7 @@ set hive.explain.user=false;
      set hive.execution.mode=llap;
      set hive.llap.execution.mode=all;
      set hive.fetch.task.conversion=none;
    -set hive.llap.daemon.allow.permanent.fns=true;
    +set hive.llap.allow.permanent.fns=true;

      drop table if exists src_orc;
      create table src_orc stored as orc as select * from src;
    @@ -37,11 +37,11 @@ DROP FUNCTION test_udf4;
      EXPLAIN
      SELECT test_udf0(cast(key as string)) from src_orc;

    -set hive.llap.daemon.allow.permanent.fns=false;
    +set hive.llap.allow.permanent.fns=false;

      EXPLAIN
      SELECT test_udf3(cast(key as string)) from src_orc;


      drop table if exists src_orc;
    -set hive.execution.mode=container;
    \ No newline at end of file
    +set hive.execution.mode=container;
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13262: LLAP: Remove log levels from DebugUtils (Prasanth Jayachandran reviewed by Sergey Shelukhin)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3b6b56d7
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3b6b56d7
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3b6b56d7

    Branch: refs/heads/llap
    Commit: 3b6b56d7000ee1d80c0f191611968d4249f311d7
    Parents: dfba1fb
    Author: Prasanth Jayachandran <prasanthj@apache.org>
    Authored: Thu Mar 24 20:49:30 2016 -0500
    Committer: Prasanth Jayachandran <prasanthj@apache.org>
    Committed: Thu Mar 24 20:49:30 2016 -0500

    ----------------------------------------------------------------------
      .../llap/IncrementalObjectSizeEstimator.java | 54 ++---
      .../hadoop/hive/llap/cache/LlapDataBuffer.java | 12 +-
      .../hive/llap/cache/LowLevelCacheImpl.java | 35 ++-
      .../llap/cache/LowLevelCacheMemoryManager.java | 6 +-
      .../llap/cache/LowLevelFifoCachePolicy.java | 4 +-
      .../llap/cache/LowLevelLrfuCachePolicy.java | 14 +-
      .../hive/llap/cache/SimpleBufferManager.java | 8 +-
      .../hive/llap/io/api/impl/LlapInputFormat.java | 32 +--
      .../hive/llap/io/api/impl/LlapIoImpl.java | 21 +-
      .../llap/io/decode/OrcColumnVectorProducer.java | 4 +-
      .../llap/io/encoded/OrcEncodedDataReader.java | 95 +++-----
      .../hadoop/hive/llap/old/BufferInProgress.java | 82 -------
      .../apache/hadoop/hive/llap/old/BufferPool.java | 225 ------------------
      .../hadoop/hive/llap/old/CachePolicy.java | 34 ---
      .../apache/hadoop/hive/llap/old/ChunkPool.java | 237 -------------------
      .../resources/llap-daemon-log4j2.properties | 14 +-
      .../org/apache/hadoop/hive/llap/DebugUtils.java | 43 ----
      .../org/apache/hadoop/hive/llap/LogLevels.java | 53 -----
      .../ql/exec/vector/VectorGroupByOperator.java | 2 +-
      .../hive/ql/io/orc/encoded/EncodedReader.java | 2 +-
      .../ql/io/orc/encoded/EncodedReaderImpl.java | 131 +++++-----
      21 files changed, 192 insertions(+), 916 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/IncrementalObjectSizeEstimator.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/IncrementalObjectSizeEstimator.java b/llap-server/src/java/org/apache/hadoop/hive/llap/IncrementalObjectSizeEstimator.java
    index 7d68294..3efbcc2 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/IncrementalObjectSizeEstimator.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/IncrementalObjectSizeEstimator.java
    @@ -186,9 +186,7 @@ public class IncrementalObjectSizeEstimator {
            fieldCol = (Collection<?>)fieldObj;
            if (fieldCol.size() == 0) {
              fieldCol = null;
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Empty collection " + field);
    - }
    + LlapIoImpl.LOG.trace("Empty collection {}", field);
            }
          }
          if (fieldCol != null) {
    @@ -219,9 +217,7 @@ public class IncrementalObjectSizeEstimator {
            fieldCol = (Map<?, ?>)fieldObj;
            if (fieldCol.size() == 0) {
              fieldCol = null;
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Empty map " + field);
    - }
    + LlapIoImpl.LOG.trace("Empty map {}", field);
            }
          }
          if (fieldCol != null) {
    @@ -257,15 +253,11 @@ public class IncrementalObjectSizeEstimator {
              return new Class<?>[] { (Class<?>)types[0], (Class<?>)types[1] };
            } else {
              // TODO: we could try to get the declaring object and infer argument... stupid Java.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Cannot determine map type: " + field);
    - }
    + LlapIoImpl.LOG.trace("Cannot determine map type: {}", field);
            }
          } else {
            // TODO: we could try to get superclass or generic interfaces.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Non-parametrized map type: " + field);
    - }
    + LlapIoImpl.LOG.trace("Non-parametrized map type: {}", field);
          }
          return null;
        }
    @@ -279,15 +271,11 @@ public class IncrementalObjectSizeEstimator {
              return (Class<?>)type;
            } else {
              // TODO: we could try to get the declaring object and infer argument... stupid Java.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Cannot determine collection type: " + field);
    - }
    + LlapIoImpl.LOG.trace("Cannot determine collection type: {}", field);
            }
          } else {
            // TODO: we could try to get superclass or generic interfaces.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Non-parametrized collection type: " + field);
    - }
    + LlapIoImpl.LOG.trace("Non-parametrized collection type: {}", field);
          }
          return null;
        }
    @@ -297,11 +285,7 @@ public class IncrementalObjectSizeEstimator {
            Field field, Object fieldObj) {
          if (fieldObj == null) return;
          int arrayLen = Array.getLength(fieldObj);
    - if (arrayLen == 0) {
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Empty array " + field);
    - }
    - }
    + LlapIoImpl.LOG.trace("Empty array {}", field);
          for (int i = 0; i < arrayLen; ++i) {
            Object element = Array.get(fieldObj, i);
            if (element != null) {
    @@ -416,10 +400,8 @@ public class IncrementalObjectSizeEstimator {
                ObjectEstimator collEstimator = parent.get(fieldObj.getClass());
                if (collEstimator == null) {
                  // We have no estimator for this type... assume low overhead and hope for the best.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Approximate estimation for collection "
    - + fieldObj.getClass().getName() + " from " + e.field);
    - }
    + LlapIoImpl.LOG.trace("Approximate estimation for collection {} from {}", e.field,
    + fieldObj.getClass().getName());
                  referencedSize += memoryModel.object();
                  referencedSize += estimateCollectionElements(parent, c, e.field, uniqueObjects);
                  referencedSize += memoryModel.array() + c.size() * memoryModel.ref();
    @@ -429,10 +411,8 @@ public class IncrementalObjectSizeEstimator {
                  referencedSize += ((CollectionEstimator)collEstimator).estimateOverhead(c.size());
                } else {
                  // We decided to treat this collection as regular object.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Verbose estimation for collection "
    - + fieldObj.getClass().getName() + " from " + e.field);
    - }
    + LlapIoImpl.LOG.trace("Verbose estimation for collection {} from {}",
    + fieldObj.getClass().getName(), e.field);
                  referencedSize += collEstimator.estimate(c, parent, uniqueObjects);
                }
                break;
    @@ -442,10 +422,8 @@ public class IncrementalObjectSizeEstimator {
                ObjectEstimator collEstimator = parent.get(fieldObj.getClass());
                if (collEstimator == null) {
                  // We have no estimator for this type... assume low overhead and hope for the best.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Approximate estimation for map "
    - + fieldObj.getClass().getName() + " from " + e.field);
    - }
    + LlapIoImpl.LOG.trace("Approximate estimation for map {} from {}",
    + fieldObj.getClass().getName(), e.field);
                  referencedSize += memoryModel.object();
                  referencedSize += estimateMapElements(parent, m, e.field, uniqueObjects);
                  referencedSize += memoryModel.array() + m.size()
    @@ -456,10 +434,8 @@ public class IncrementalObjectSizeEstimator {
                  referencedSize += ((CollectionEstimator)collEstimator).estimateOverhead(m.size());
                } else {
                  // We decided to treat this map as regular object.
    - if (DebugUtils.isTraceEnabled()) {
    - LlapIoImpl.LOG.info("Verbose estimation for map "
    - + fieldObj.getClass().getName() + " from " + e.field);
    - }
    + LlapIoImpl.LOG.trace("Verbose estimation for map {} from {}",
    + fieldObj.getClass().getName(), e.field);
                  referencedSize += collEstimator.estimate(m, parent, uniqueObjects);
                }
                break;

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapDataBuffer.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapDataBuffer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapDataBuffer.java
    index 840aeab..d1a961c 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapDataBuffer.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LlapDataBuffer.java
    @@ -83,8 +83,8 @@ public final class LlapDataBuffer extends LlapCacheableBuffer implements MemoryB
            newRefCount = oldRefCount + 1;
            if (refCount.compareAndSet(oldRefCount, newRefCount)) break;
          }
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Locked " + this + "; new ref count " + newRefCount);
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Locked {}; new ref count {}", this, newRefCount);
          }
          return newRefCount;
        }
    @@ -109,8 +109,8 @@ public final class LlapDataBuffer extends LlapCacheableBuffer implements MemoryB

        int decRef() {
          int newRefCount = refCount.decrementAndGet();
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Unlocked " + this + "; refcount " + newRefCount);
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Unlocked {}; refcount {}", this, newRefCount);
          }
          if (newRefCount < 0) {
            throw new AssertionError("Unexpected refCount " + newRefCount + ": " + this);
    @@ -128,8 +128,8 @@ public final class LlapDataBuffer extends LlapCacheableBuffer implements MemoryB
            if (value != 0) return false;
            if (refCount.compareAndSet(value, EVICTED_REFCOUNT)) break;
          }
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Invalidated " + this + " due to eviction");
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Invalidated {} due to eviction", this);
          }
          return true;
        }

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
    index a60fed3..038c3ed 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheImpl.java
    @@ -58,9 +58,8 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
        @VisibleForTesting
        LowLevelCacheImpl(LlapDaemonCacheMetrics metrics, LowLevelCachePolicy cachePolicy,
            EvictionAwareAllocator allocator, boolean doAssumeGranularBlocks, long cleanupInterval) {
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("Low level cache; cleanup interval " + cleanupInterval + "sec");
    - }
    +
    + LlapIoImpl.LOG.info("Low level cache; cleanup interval {} sec", cleanupInterval);
          this.cachePolicy = cachePolicy;
          this.allocator = allocator;
          this.cleanupInterval = cleanupInterval;
    @@ -148,8 +147,8 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
            LlapDataBuffer buffer = e.getValue();
            long requestedLength = currentNotCached.getLength();
            // Lock the buffer, validate it and add to results.
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Locking " + buffer + " during get");
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Locking {} during get", buffer);
            }

            if (!lockBuffer(buffer, true)) {
    @@ -183,7 +182,6 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
         * Adds cached buffer to buffer list.
         * @param currentNotCached Pointer to the list node where we are inserting.
         * @param currentCached The cached buffer found for this node, to insert.
    - * @param resultObj
         * @return The new currentNotCached pointer, following the cached buffer insertion.
         */
        private DiskRangeList addCachedBufferToIter(
    @@ -240,8 +238,8 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
          try {
            for (int i = 0; i < ranges.length; ++i) {
              LlapDataBuffer buffer = (LlapDataBuffer)buffers[i];
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Locking " + buffer + " at put time");
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Locking {} at put time", buffer);
              }
              boolean canLock = lockBuffer(buffer, false);
              assert canLock;
    @@ -258,13 +256,13 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
                  }
                  break;
                }
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Trying to cache when the chunk is already cached for "
    - + fileKey + "@" + offset + " (base " + baseOffset + "); old " + oldVal
    - + ", new " + buffer);
    + if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.CACHE_LOGGER.trace("Trying to cache when the chunk is already cached for" +
    + " {}@{} (base {}); old {}, new {}", fileKey, offset, baseOffset, oldVal, buffer);
                }
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Locking " + oldVal + " due to cache collision");
    +
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Locking {} due to cache collision", oldVal);
                }
                if (lockBuffer(oldVal, true)) {
                  // We don't do proper overlap checking because it would cost cycles and we
    @@ -275,8 +273,9 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
                        + " (base " + baseOffset + ")");
                  }
                  // We found an old, valid block for this key in the cache.
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Unlocking " + buffer + " due to cache collision with " + oldVal);
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.LOCKING_LOGGER.trace("Unlocking {} due to cache collision with {}",
    + buffer, oldVal);
                  }

                  unlockBuffer(buffer, false);
    @@ -353,8 +352,8 @@ public class LowLevelCacheImpl implements LowLevelCache, BufferUsageManager, Lla
            if (buffer.declaredCachedLength != LlapDataBuffer.UNKNOWN_CACHED_LENGTH) {
              cachePolicy.notifyUnlock(buffer);
            } else {
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Deallocating " + buffer + " that was not cached");
    + if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.CACHE_LOGGER.trace("Deallocating {} that was not cached", buffer);
              }
              allocator.deallocate(buffer);
            }

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
    index 1cfe2bc..4def4a1 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelCacheMemoryManager.java
    @@ -50,9 +50,9 @@ public class LowLevelCacheMemoryManager implements MemoryManager {
          this.usedMemory = new AtomicLong(0);
          this.metrics = metrics;
          metrics.setCacheCapacityTotal(maxSize);
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("Memory manager initialized with max size " + maxSize + " and "
    - + ((evictor == null) ? "no " : "") + "ability to evict blocks");
    + if (LlapIoImpl.LOG.isInfoEnabled()) {
    + LlapIoImpl.LOG.info("Memory manager initialized with max size {} and" +
    + " {} ability to evict blocks", maxSize, ((evictor == null) ? "no " : ""));
          }
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
    index 1430eae..0838682 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelFifoCachePolicy.java
    @@ -35,9 +35,7 @@ public class LowLevelFifoCachePolicy implements LowLevelCachePolicy {
        private LlapOomDebugDump parentDebugDump;

        public LowLevelFifoCachePolicy(Configuration conf) {
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("FIFO cache policy");
    - }
    + LlapIoImpl.LOG.info("FIFO cache policy");
          buffers = new LinkedList<LlapCacheableBuffer>();
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
    index 6f52b86..bbff3cc 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/LowLevelLrfuCachePolicy.java
    @@ -82,10 +82,8 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
            int lrfuThreshold = (int)((Math.log(1 - Math.pow(0.5, lambda)) / Math.log(0.5)) / lambda);
            maxHeapSize = Math.min(lrfuThreshold, maxBuffers);
          }
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("LRFU cache policy with min buffer size " + minBufferSize
    - + " and lambda " + lambda + " (heap size " + maxHeapSize + ")");
    - }
    + LlapIoImpl.LOG.info("LRFU cache policy with min buffer size {} and lambda {} (heap size {})",
    + minBufferSize, lambda, maxHeapSize);

          heap = new LlapCacheableBuffer[maxHeapSize];
          listHead = listTail = null;
    @@ -123,8 +121,8 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
        @Override
        public void notifyUnlock(LlapCacheableBuffer buffer) {
          long time = timer.incrementAndGet();
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Touching " + buffer + " at " + time);
    + if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.CACHE_LOGGER.trace("Touching {} at {}", buffer, time);
          }
          synchronized (heap) {
            // First, update buffer priority - we have just been using it.
    @@ -263,8 +261,8 @@ public class LowLevelLrfuCachePolicy implements LowLevelCachePolicy {
          while (true) {
            if (heapSize == 0) return null;
            LlapCacheableBuffer result = heap[0];
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Evicting " + result + " at " + time);
    + if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.CACHE_LOGGER.info("Evicting {} at {}", result, time);
            }
            result.indexInHeap = LlapCacheableBuffer.NOT_IN_CACHE;
            --heapSize;

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
    index 734a5c0..b188c0e 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/cache/SimpleBufferManager.java
    @@ -30,9 +30,7 @@ public class SimpleBufferManager implements BufferUsageManager {
        private final LlapDaemonCacheMetrics metrics;

        public SimpleBufferManager(Allocator allocator, LlapDaemonCacheMetrics metrics) {
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("Simple buffer manager");
    - }
    + LlapIoImpl.LOG.info("Simple buffer manager");
          this.allocator = allocator;
          this.metrics = metrics;
        }
    @@ -46,8 +44,8 @@ public class SimpleBufferManager implements BufferUsageManager {

        private void unlockBuffer(LlapDataBuffer buffer) {
          if (buffer.decRef() == 0) {
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Deallocating " + buffer + " that was not cached");
    + if (LlapIoImpl.CACHE_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.CACHE_LOGGER.trace("Deallocating {} that was not cached", buffer);
            }
            allocator.deallocate(buffer);
          }

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
    index 85cca97..9fb79a5 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapInputFormat.java
    @@ -264,21 +264,21 @@ public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowB
            }
            synchronized (pendingData) {
              // We are waiting for next block. Either we will get it, or be told we are done.
    - boolean doLogBlocking = DebugUtils.isTraceMttEnabled() && isNothingToReport();
    + boolean doLogBlocking = LlapIoImpl.LOG.isTraceEnabled() && isNothingToReport();
              if (doLogBlocking) {
    - LlapIoImpl.LOG.info("next will block");
    + LlapIoImpl.LOG.trace("next will block");
              }
              while (isNothingToReport()) {
                pendingData.wait(100);
              }
              if (doLogBlocking) {
    - LlapIoImpl.LOG.info("next is unblocked");
    + LlapIoImpl.LOG.trace("next is unblocked");
              }
              rethrowErrorIfAny();
              lastCvb = pendingData.poll();
            }
    - if (DebugUtils.isTraceMttEnabled() && lastCvb != null) {
    - LlapIoImpl.LOG.info("Processing will receive vector " + lastCvb);
    + if (LlapIoImpl.LOG.isTraceEnabled() && lastCvb != null) {
    + LlapIoImpl.LOG.trace("Processing will receive vector {}", lastCvb);
            }
            return lastCvb;
          }
    @@ -304,9 +304,9 @@ public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowB

          @Override
          public void close() throws IOException {
    - if (DebugUtils.isTraceMttEnabled()) {
    - LlapIoImpl.LOG.info("close called; closed " + isClosed + ", done " + isDone
    - + ", err " + pendingError + ", pending " + pendingData.size());
    + if (LlapIoImpl.LOG.isTraceEnabled()) {
    + LlapIoImpl.LOG.trace("close called; closed {}, done {}, err {}, pending {}",
    + isClosed, isDone, pendingError, pendingData.size());
            }
            LlapIoImpl.LOG.info("Llap counters: {}" ,counters); // This is where counters are logged!
            feedback.stop();
    @@ -323,9 +323,9 @@ public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowB

          @Override
          public void setDone() {
    - if (DebugUtils.isTraceMttEnabled()) {
    - LlapIoImpl.LOG.info("setDone called; closed " + isClosed
    - + ", done " + isDone + ", err " + pendingError + ", pending " + pendingData.size());
    + if (LlapIoImpl.LOG.isTraceEnabled()) {
    + LlapIoImpl.LOG.trace("setDone called; closed {}, done {}, err {}, pending {}",
    + isClosed, isDone, pendingError, pendingData.size());
            }
            synchronized (pendingData) {
              isDone = true;
    @@ -335,9 +335,9 @@ public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowB

          @Override
          public void consumeData(ColumnVectorBatch data) {
    - if (DebugUtils.isTraceMttEnabled()) {
    - LlapIoImpl.LOG.info("consume called; closed " + isClosed + ", done " + isDone
    - + ", err " + pendingError + ", pending " + pendingData.size());
    + if (LlapIoImpl.LOG.isTraceEnabled()) {
    + LlapIoImpl.LOG.trace("consume called; closed {}, done {}, err {}, pending {}",
    + isClosed, isDone, pendingError, pendingData.size());
            }
            synchronized (pendingData) {
              if (isClosed) {
    @@ -351,8 +351,8 @@ public class LlapInputFormat implements InputFormat<NullWritable, VectorizedRowB
          @Override
          public void setError(Throwable t) {
            counters.incrCounter(LlapIOCounters.NUM_ERRORS);
    - LlapIoImpl.LOG.info("setError called; closed " + isClosed
    - + ", done " + isDone + ", err " + pendingError + ", pending " + pendingData.size());
    + LlapIoImpl.LOG.info("setError called; closed {}, done {}, err {}, pending {}",
    + isClosed, isDone, pendingError, pendingData.size());
            assert t != null;
            synchronized (pendingData) {
              pendingError = t;

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
    index dbee823..36f8dec 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/api/impl/LlapIoImpl.java
    @@ -18,8 +18,6 @@

      package org.apache.hadoop.hive.llap.io.api.impl;

    -import org.apache.hadoop.hive.llap.LogLevels;
    -
      import java.io.IOException;
      import java.util.concurrent.Executors;

    @@ -58,8 +56,11 @@ import com.google.common.util.concurrent.MoreExecutors;
      import com.google.common.util.concurrent.ThreadFactoryBuilder;

      public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
    - public static final Logger LOG = LoggerFactory.getLogger(LlapIoImpl.class);
    - public static final LogLevels LOGL = new LogLevels(LOG);
    + public static final Logger LOG = LoggerFactory.getLogger("LlapIoImpl");
    + public static final Logger ORC_LOGGER = LoggerFactory.getLogger("LlapIoOrc");
    + public static final Logger CACHE_LOGGER = LoggerFactory.getLogger("LlapIoCache");
    + public static final Logger LOCKING_LOGGER = LoggerFactory.getLogger("LlapIoLocking");
    +
        private static final String MODE_CACHE = "cache", MODE_ALLOCATOR = "allocator";

        private final ColumnVectorProducer cvp;
    @@ -73,9 +74,7 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
          String ioMode = HiveConf.getVar(conf, HiveConf.ConfVars.LLAP_IO_MEMORY_MODE);
          boolean useLowLevelCache = LlapIoImpl.MODE_CACHE.equalsIgnoreCase(ioMode),
              useAllocOnly = !useLowLevelCache && LlapIoImpl.MODE_ALLOCATOR.equalsIgnoreCase(ioMode);
    - if (LOGL.isInfoEnabled()) {
    - LOG.info("Initializing LLAP IO in " + ioMode + " mode");
    - }
    + LOG.info("Initializing LLAP IO in {} mode", ioMode);

          String displayName = "LlapDaemonCacheMetrics-" + MetricsUtils.getHostName();
          String sessionId = conf.get("llap.daemon.metrics.sessionid");
    @@ -86,8 +85,8 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
              HiveConf.ConfVars.LLAP_QUEUE_METRICS_PERCENTILE_INTERVALS));
          this.queueMetrics = LlapDaemonQueueMetrics.create(displayName, sessionId, intervals);

    - LOG.info("Started llap daemon metrics with displayName: " + displayName +
    - " sessionId: " + sessionId);
    + LOG.info("Started llap daemon metrics with displayName: {} sessionId: {}", displayName,
    + sessionId);

          OrcMetadataCache metadataCache = null;
          LowLevelCacheImpl orcCache = null;
    @@ -128,9 +127,7 @@ public class LlapIoImpl implements LlapIo<VectorizedRowBatch> {
          // TODO: this should depends on input format and be in a map, or something.
          this.cvp = new OrcColumnVectorProducer(
              metadataCache, orcCache, bufferManager, conf, cacheMetrics, queueMetrics);
    - if (LOGL.isInfoEnabled()) {
    - LOG.info("LLAP IO initialized");
    - }
    + LOG.info("LLAP IO initialized");

          registerMXBeans();
        }

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
    index 37fc8d0..024c485 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/decode/OrcColumnVectorProducer.java
    @@ -49,9 +49,7 @@ public class OrcColumnVectorProducer implements ColumnVectorProducer {
        public OrcColumnVectorProducer(OrcMetadataCache metadataCache,
            LowLevelCacheImpl lowLevelCache, BufferUsageManager bufferManager,
            Configuration conf, LlapDaemonCacheMetrics metrics, LlapDaemonQueueMetrics queueMetrics) {
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("Initializing ORC column vector producer");
    - }
    + LlapIoImpl.LOG.info("Initializing ORC column vector producer");

          this.metadataCache = metadataCache;
          this.lowLevelCache = lowLevelCache;

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
    index eb251a8..fb0867d 100644
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
    +++ b/llap-server/src/java/org/apache/hadoop/hive/llap/io/encoded/OrcEncodedDataReader.java
    @@ -184,9 +184,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>

        @Override
        public void stop() {
    - if (LOG.isDebugEnabled()) {
    - LOG.debug("Encoded reader is being stopped");
    - }
    + LOG.debug("Encoded reader is being stopped");
          isStopped = true;
        }

    @@ -214,9 +212,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>

        protected Void performDataRead() throws IOException {
          long startTime = counters.startTimeCounter();
    - if (LlapIoImpl.LOGL.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("Processing data for " + split.getPath());
    - }
    + LlapIoImpl.LOG.info("Processing data for {}", split.getPath());
          if (processStop()) {
            recordReaderTime(startTime);
            return null;
    @@ -310,7 +306,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
            // Reader creating updates HDFS counters, don't do it here.
            DataWrapperForOrc dw = new DataWrapperForOrc();
            stripeReader = orcReader.encodedReader(fileKey, dw, dw, POOL_FACTORY);
    - stripeReader.setDebugTracing(DebugUtils.isTraceOrcEnabled());
    + stripeReader.setTracing(LlapIoImpl.ORC_LOGGER.isTraceEnabled());
          } catch (Throwable t) {
            consumer.setError(t);
            recordReaderTime(startTime);
    @@ -338,10 +334,8 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
              if (cols != null && cols.isEmpty()) continue; // No need to read this stripe.
              stripe = fileMetadata.getStripes().get(stripeIx);

    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Reading stripe " + stripeIx + ": "
    - + stripe.getOffset() + ", " + stripe.getLength());
    - }
    + LlapIoImpl.ORC_LOGGER.trace("Reading stripe {}: {}, {}", stripeIx, stripe.getOffset(),
    + stripe.getLength());
              colRgs = readState[stripeIxMod];
              // We assume that NO_RGS value is only set from SARG filter and for all columns;
              // intermediate changes for individual columns will unset values in the array.
    @@ -379,18 +373,18 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
                  counters.incrTimeCounter(LlapIOCounters.HDFS_TIME_NS, startTimeHdfs);
                  if (hasFileId && metadataCache != null) {
                    stripeMetadata = metadataCache.putStripeMetadata(stripeMetadata);
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Caching stripe " + stripeKey.stripeIx
    - + " metadata with includes: " + DebugUtils.toString(stripeIncludes));
    + if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.ORC_LOGGER.trace("Caching stripe {} metadata with includes: {}",
    + stripeKey.stripeIx, DebugUtils.toString(stripeIncludes));
                    }
                  }
                }
                consumer.setStripeMetadata(stripeMetadata);
              }
              if (!stripeMetadata.hasAllIndexes(stripeIncludes)) {
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Updating indexes in stripe " + stripeKey.stripeIx
    - + " metadata for includes: " + DebugUtils.toString(stripeIncludes));
    + if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.ORC_LOGGER.trace("Updating indexes in stripe {} metadata for includes: {}",
    + stripeKey.stripeIx, DebugUtils.toString(stripeIncludes));
                }
                assert isFoundInCache;
                counters.incrCounter(LlapIOCounters.METADATA_CACHE_MISS);
    @@ -432,9 +426,8 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
          // Done with all the things.
          recordReaderTime(startTime);
          consumer.setDone();
    - if (DebugUtils.isTraceMttEnabled()) {
    - LlapIoImpl.LOG.info("done processing " + split);
    - }
    +
    + LlapIoImpl.LOG.trace("done processing {}", split);

          // Close the stripe reader, we are done reading.
          cleanupReaders();
    @@ -584,9 +577,7 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
          if (fileKey instanceof Long && HiveConf.getBoolVar(conf, ConfVars.LLAP_IO_USE_FILEID_PATH)) {
            path = HdfsUtils.getFileIdPath(fs, path, (long)fileKey);
          }
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LOG.info("Creating reader for " + path + " (" + split.getPath() + ")");
    - }
    + LlapIoImpl.ORC_LOGGER.trace("Creating reader for {} ({})", path, split.getPath());
          long startTime = counters.startTimeCounter();
          ReaderOptions opts = OrcFile.readerOptions(conf).filesystem(fs).fileMetadata(fileMetadata);
          orcReader = EncodedOrcFile.createReader(path, opts);
    @@ -640,17 +631,17 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
                counters.incrTimeCounter(LlapIOCounters.HDFS_TIME_NS, startTime);
                if (hasFileId && metadataCache != null) {
                  value = metadataCache.putStripeMetadata(value);
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Caching stripe " + stripeKey.stripeIx
    - + " metadata with includes: " + DebugUtils.toString(globalInc));
    + if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.ORC_LOGGER.trace("Caching stripe {} metadata with includes: {}",
    + stripeKey.stripeIx, DebugUtils.toString(globalInc));
                  }
                }
              }
              // We might have got an old value from cache; recheck it has indexes.
              if (!value.hasAllIndexes(globalInc)) {
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Updating indexes in stripe " + stripeKey.stripeIx
    - + " metadata for includes: " + DebugUtils.toString(globalInc));
    + if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.ORC_LOGGER.trace("Updating indexes in stripe {} metadata for includes: {}",
    + stripeKey.stripeIx, DebugUtils.toString(globalInc));
                }
                updateLoadedIndexes(value, si, globalInc, sargColumns);
              }
    @@ -677,9 +668,9 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
            if (datas == null) continue;
            for (ColumnStreamData data : datas) {
              if (data == null || data.decRef() != 0) continue;
    - if (DebugUtils.isTraceLockingEnabled()) {
    + if (LlapIoImpl.LOCKING_LOGGER.isTraceEnabled()) {
                for (MemoryBuffer buf : data.getCacheBuffers()) {
    - LlapIoImpl.LOG.info("Unlocking " + buf + " at the end of processing");
    + LlapIoImpl.LOCKING_LOGGER.trace("Unlocking {} at the end of processing", buf);
                }
              }
              bufferManager.decRefBuffers(data.getCacheBuffers());
    @@ -718,14 +709,14 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
            boolean isNone = rgsToRead == SargApplier.READ_NO_RGS,
                isAll = rgsToRead == SargApplier.READ_ALL_RGS;
            hasAnyData = hasAnyData || !isNone;
    - if (DebugUtils.isTraceOrcEnabled()) {
    + if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
              if (isNone) {
    - LlapIoImpl.LOG.info("SARG eliminated all RGs for stripe " + stripeIx);
    + LlapIoImpl.ORC_LOGGER.trace("SARG eliminated all RGs for stripe {}", stripeIx);
              } else if (!isAll) {
    - LlapIoImpl.LOG.info("SARG picked RGs for stripe " + stripeIx + ": "
    - + DebugUtils.toString(rgsToRead));
    + LlapIoImpl.ORC_LOGGER.trace("SARG picked RGs for stripe {}: {}",
    + stripeIx, DebugUtils.toString(rgsToRead));
              } else {
    - LlapIoImpl.LOG.info("Will read all " + rgCount + " RGs for stripe " + stripeIx);
    + LlapIoImpl.ORC_LOGGER.trace("Will read all {} RGs for stripe {}", rgCount, stripeIx);
              }
            }
            assert isAll || isNone || rgsToRead.length == rgCount;
    @@ -768,12 +759,12 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
          long offset = split.getStart(), maxOffset = offset + split.getLength();
          stripeIxFrom = -1;
          int stripeIxTo = -1;
    - if (LlapIoImpl.LOGL.isDebugEnabled()) {
    + if (LlapIoImpl.ORC_LOGGER.isDebugEnabled()) {
            String tmp = "FileSplit {" + split.getStart() + ", " + split.getLength() + "}; stripes ";
            for (StripeInformation stripe : stripes) {
              tmp += "{" + stripe.getOffset() + ", " + stripe.getLength() + "}, ";
            }
    - LlapIoImpl.LOG.debug(tmp);
    + LlapIoImpl.ORC_LOGGER.debug(tmp);
          }

          int stripeIx = 0;
    @@ -785,33 +776,25 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
              continue;
            }
            if (stripeIxFrom == -1) {
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Including stripes from " + stripeIx
    - + " (" + stripeStart + " >= " + offset + ")");
    - }
    + LlapIoImpl.ORC_LOGGER.trace("Including stripes from {} ({} >= {})",
    + stripeIx, stripeStart, offset);
              stripeIxFrom = stripeIx;
            }
            if (stripeStart >= maxOffset) {
              stripeIxTo = stripeIx;
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Including stripes until " + stripeIxTo + " (" + stripeStart
    - + " >= " + maxOffset + "); " + (stripeIxTo - stripeIxFrom) + " stripes");
    - }
    + LlapIoImpl.ORC_LOGGER.trace("Including stripes until {} ({} >= {}); {} stripes",
    + stripeIxTo, stripeStart, maxOffset, (stripeIxTo - stripeIxFrom));
              break;
            }
            ++stripeIx;
          }
          if (stripeIxFrom == -1) {
    - if (LlapIoImpl.LOG.isInfoEnabled()) {
    - LlapIoImpl.LOG.info("Not including any stripes - empty split");
    - }
    + LlapIoImpl.LOG.info("Not including any stripes - empty split");
          }
          if (stripeIxTo == -1 && stripeIxFrom != -1) {
            stripeIxTo = stripeIx;
    - if (DebugUtils.isTraceOrcEnabled()) {
    - LlapIoImpl.LOG.info("Including stripes until " + stripeIx + " (end of file); "
    - + (stripeIxTo - stripeIxFrom) + " stripes");
    - }
    + LlapIoImpl.ORC_LOGGER.trace("Including stripes until {} (end of file); {} stripes",
    + stripeIx, (stripeIxTo - stripeIxFrom));
          }
          readState = new boolean[stripeIxTo - stripeIxFrom][][];
        }
    @@ -869,9 +852,9 @@ public class OrcEncodedDataReader extends CallableWithNdc<Void>
            long startTime = counters.startTimeCounter();
            DiskRangeList result = orcDataReader.readFileData(range, baseOffset, doForceDirect);
            counters.recordHdfsTime(startTime);
    - if (DebugUtils.isTraceOrcEnabled() && LOG.isInfoEnabled()) {
    - LOG.info("Disk ranges after disk read (file " + fileKey + ", base offset " + baseOffset
    - + "): " + RecordReaderUtils.stringifyDiskRanges(result));
    + if (LlapIoImpl.ORC_LOGGER.isTraceEnabled()) {
    + LlapIoImpl.ORC_LOGGER.trace("Disk ranges after disk read (file {}, base offset {}): {}",
    + fileKey, baseOffset, RecordReaderUtils.stringifyDiskRanges(result));
            }
            return result;
          }

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferInProgress.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferInProgress.java b/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferInProgress.java
    deleted file mode 100644
    index 9782b81..0000000
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferInProgress.java
    +++ /dev/null
    @@ -1,82 +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.hadoop.hive.llap.old;
    -
    -import org.apache.hadoop.hive.llap.old.BufferPool.WeakBuffer;
    -import org.apache.hadoop.hive.llap.old.ChunkPool.Chunk;
    -
    -/**
    - * Helper struct that is used by loaders (e.g. OrcLoader) and chunk writer to write chunks.
    - */
    -public class BufferInProgress {
    - /** Buffer that is being written to. */
    - public final WeakBuffer buffer;
    - /** Offset in buffer where writing can proceed */
    - public int offset; // TODO: use WB's position; these have separate lifecycle now, needed?
    - private final int bufferLimit;
    -
    - /** The chunk that is currently being written. */
    - private Chunk chunkInProgress = null;
    - /** The row count of the chunk currently being written. */
    - private int chunkInProgressRows = 0;
    -
    - public BufferInProgress(WeakBuffer buffer) {
    - this.buffer = buffer;
    - this.bufferLimit = buffer.getContents().limit();
    - this.offset = 0;
    - }
    -
    - public Chunk ensureChunk() {
    - if (chunkInProgress == null) {
    - chunkInProgress = new Chunk(buffer, offset, 0);
    - chunkInProgressRows = 0;
    - }
    - return chunkInProgress;
    - }
    -
    - public Chunk extractChunk() {
    - Chunk result = chunkInProgress;
    - chunkInProgress = null;
    - chunkInProgressRows = 0;
    - return result;
    - }
    -
    - public void update(int newOffset, int rowsWritten) {
    - if (newOffset > bufferLimit) {
    - throw new AssertionError("Offset is beyond buffer limit: " + newOffset + "/" + bufferLimit
    - + "; previous offset " + offset + ", chunk " + chunkInProgress);
    - }
    - chunkInProgress.length += (newOffset - offset);
    - this.offset = newOffset;
    - this.chunkInProgressRows += rowsWritten;
    - }
    -
    - public int getChunkInProgressRows() {
    - return chunkInProgressRows;
    - }
    -
    - public int getSpaceLeft() {
    - return getSpaceLeft(-1);
    - }
    -
    - public int getSpaceLeft(int offset) {
    - offset = (offset >= 0) ? offset : this.offset;
    - return buffer.getContents().limit() - offset;
    - }
    -}

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferPool.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferPool.java b/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferPool.java
    deleted file mode 100644
    index fc10b2b..0000000
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/old/BufferPool.java
    +++ /dev/null
    @@ -1,225 +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.hadoop.hive.llap.old;
    -
    -import java.nio.ByteBuffer;
    -import java.util.concurrent.atomic.AtomicInteger;
    -
    -import org.apache.hadoop.conf.Configuration;
    -import org.apache.hadoop.hive.conf.HiveConf;
    -import org.apache.hadoop.hive.llap.DebugUtils;
    -import org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl;
    -
    -import com.google.common.annotations.VisibleForTesting;
    -
    -public class BufferPool {
    - // TODO: we should keep evicted buffers for reuse. Perhaps that too should be factored out.
    - private final CachePolicy cachePolicy;
    - private final Object evictionNotifyObj = new Object();
    - private int evictionIsWaiting; // best effort flag
    - private final long maxCacheSize;
    - private final int bufferSize;
    -
    -
    - public BufferPool(Configuration conf) {
    - this.maxCacheSize = 0;// HiveConf.getLongVar(conf, HiveConf.ConfVars.LLAP_CACHE_SIZE);
    - this.bufferSize = 0; // HiveConf.getIntVar(conf, HiveConf.ConfVars.LLAP_BUFFER_SIZE);
    - this.cachePolicy = null;
    - }
    -
    - /**
    - * Allocates a new buffer. Buffer starts out locked (assumption is that caller is going to
    - * write to it immediately and then unlock it; future writers/readers will lock and unlock).
    - * @return Buffer.
    - */
    - public WeakBuffer allocateBuffer() throws InterruptedException {
    - // TODO: for now, dumb byte arrays. Should be off-heap.
    - ByteBuffer newBuffer = ByteBuffer.allocate(bufferSize);
    - WeakBuffer wb = new WeakBuffer(this, newBuffer);
    - // Don't touch the buffer - it's not in cache yet. cache() will set the initial priority.
    - if (!wb.lock(false)) {
    - throw new AssertionError("Cannot lock a new buffer");
    - }
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Locked " + wb + " after creation");
    - }
    - boolean hasWaited = false;
    - WeakBuffer evicted = null;
    - while (true) {
    - evicted = cachePolicy.cache(wb);
    - if (evicted != CachePolicy.CANNOT_EVICT) break;
    - if (DebugUtils.isTraceCachingEnabled() && !hasWaited) {
    - LlapIoImpl.LOG.info("Failed to add a new block to cache; waiting for blocks to be unlocked");
    - hasWaited = true;
    - }
    - synchronized (evictionNotifyObj) {
    - ++evictionIsWaiting;
    - evictionNotifyObj.wait(1000);
    - --evictionIsWaiting;
    - }
    - }
    - if (DebugUtils.isTraceCachingEnabled() && hasWaited) {
    - LlapIoImpl.LOG.info("Eviction is done waiting");
    - }
    - if (evicted != null) {
    - //if (evictionListener != null) {
    - // evictionListener.evictionNotice(evicted);
    - //}
    - // After eviction notice, the contents can be reset.
    - evicted.clear();
    - }
    - return wb;
    - }
    -
    - private final void unblockEviction() {
    - if (evictionIsWaiting <= 0) return;
    - synchronized (evictionNotifyObj) {
    - if (evictionIsWaiting <= 0) return;
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Notifying eviction that some block has been unlocked");
    - }
    - evictionNotifyObj.notifyAll();
    - }
    - }
    -
    - @VisibleForTesting
    - public static WeakBuffer allocateFake() {
    - return new WeakBuffer(null, ByteBuffer.wrap(new byte[1]));
    - }
    -
    - /**
    - * This class serves 3 purposes:
    - * 1) it implements BufferPool-specific hashCode and equals (ByteBuffer ones are content-based);
    - * 2) it contains the refCount;
    - * 3) by extension from (2), it can be held while it is evicted; when locking before the usage,
    - * the fact that the data has been evicted will be discovered (similar to weak_ptr).
    - * Note: not static because when we wait for something to become evict-able,
    - * we need to receive notifications from unlock (see unlock). Otherwise could be static.
    - */
    - public static final class WeakBuffer {
    - private static final int EVICTED_REFCOUNT = -1;
    - private final BufferPool parent;
    - private ByteBuffer contents;
    - private final AtomicInteger refCount = new AtomicInteger(0);
    -
    - // TODO: Fields pertaining to cache policy. Perhaps they should live in separate object.
    - public double priority;
    - public long lastUpdate = -1;
    - public int indexInHeap = -1;
    - public boolean isLockedInHeap = false;
    -
    - private WeakBuffer(BufferPool parent, ByteBuffer contents) {
    - this.parent = parent;
    - this.contents = contents;
    - }
    -
    - public ByteBuffer getContents() {
    - assert isLocked() : "Cannot get contents with refCount " + refCount.get();
    - return contents;
    - }
    -
    - @Override
    - public int hashCode() {
    - if (contents == null) return 0;
    - return System.identityHashCode(contents);
    - }
    -
    - @Override
    - public boolean equals(Object obj) {
    - if (this == obj) return true;
    - if (!(obj instanceof WeakBuffer)) return false;
    - // We only compare objects, and not contents of the ByteBuffer.
    - // One ByteBuffer is never put in multiple WeakBuffer-s (that is the invariant).
    - return contents == ((WeakBuffer)obj).contents;
    - }
    -
    - public boolean lock(boolean doTouch) {
    - int oldRefCount = -1;
    - while (true) {
    - oldRefCount = refCount.get();
    - if (oldRefCount == EVICTED_REFCOUNT) return false;
    - assert oldRefCount >= 0;
    - if (refCount.compareAndSet(oldRefCount, oldRefCount + 1)) break;
    - }
    - if (doTouch && oldRefCount == 0 && parent != null) {
    - parent.cachePolicy.notifyLock(this);
    - }
    - return true;
    - }
    -
    - public boolean isLocked() {
    - // Best-effort check. We cannot do a good check against caller thread, since
    - // refCount could still be > 0 if someone else locked. This is used for asserts.
    - return refCount.get() > 0;
    - }
    -
    - public boolean isInvalid() {
    - return refCount.get() == EVICTED_REFCOUNT;
    - }
    -
    - public boolean isCleared() {
    - return contents == null;
    - }
    -
    - public void unlock() {
    - int newRefCount = refCount.decrementAndGet();
    - if (newRefCount < 0) {
    - throw new AssertionError("Unexpected refCount " + newRefCount);
    - }
    - // If this block became eligible, see if we need to unblock the eviction.
    - if (newRefCount == 0 && parent != null) {
    - parent.cachePolicy.notifyUnlock(this);
    - parent.unblockEviction();
    - }
    - }
    -
    - @Override
    - public String toString() {
    - return "0x" + Integer.toHexString(hashCode());
    - }
    -
    - /**
    - * @return Whether the we can invalidate; false if locked or already evicted.
    - */
    - boolean invalidate() {
    - while (true) {
    - int value = refCount.get();
    - if (value != 0) return false;
    - if (refCount.compareAndSet(value, EVICTED_REFCOUNT)) break;
    - }
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Invalidated " + this + " due to eviction");
    - }
    - return true;
    - }
    -
    - ByteBuffer clear() {
    - assert refCount.get() == EVICTED_REFCOUNT;
    - ByteBuffer result = contents;
    - contents = null;
    - return result;
    - }
    -
    - public String toStringForCache() {
    - return "[" + Integer.toHexString(hashCode()) + " " + String.format("%1$.2f", priority) + " "
    - + lastUpdate + " " + (isLocked() ? "!" : ".") + "]";
    - }
    - }
    -}

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/old/CachePolicy.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/old/CachePolicy.java b/llap-server/src/java/org/apache/hadoop/hive/llap/old/CachePolicy.java
    deleted file mode 100644
    index cca42fe..0000000
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/old/CachePolicy.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.hadoop.hive.llap.old;
    -
    -import org.apache.hadoop.hive.llap.old.BufferPool.WeakBuffer;
    -
    -public interface CachePolicy {
    - public static final WeakBuffer CANNOT_EVICT = BufferPool.allocateFake();
    -
    - /**
    - * @param buffer Buffer to cache.
    - * @return Evicted buffer. All buffers are of the same size currently, so it is one or none.
    - * It can also be CANNOT_EVICT fake buffer, if we cannot evict and thus cache.
    - */
    - WeakBuffer cache(WeakBuffer buffer);
    - void notifyLock(WeakBuffer buffer);
    - void notifyUnlock(WeakBuffer buffer);
    -}

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/java/org/apache/hadoop/hive/llap/old/ChunkPool.java
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/java/org/apache/hadoop/hive/llap/old/ChunkPool.java b/llap-server/src/java/org/apache/hadoop/hive/llap/old/ChunkPool.java
    deleted file mode 100644
    index 4f9f165..0000000
    --- a/llap-server/src/java/org/apache/hadoop/hive/llap/old/ChunkPool.java
    +++ /dev/null
    @@ -1,237 +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.hadoop.hive.llap.old;
    -
    -import java.util.HashSet;
    -import java.util.Iterator;
    -import java.util.List;
    -import java.util.Map;
    -import java.util.concurrent.ConcurrentHashMap;
    -import java.util.concurrent.atomic.AtomicInteger;
    -
    -import org.apache.hadoop.hive.llap.DebugUtils;
    -import org.apache.hadoop.hive.llap.io.api.impl.LlapIoImpl;
    -import org.apache.hadoop.hive.llap.old.BufferPool.WeakBuffer;
    -import org.apache.hadoop.hive.llap.old.ChunkPool.Chunk;
    -
    -/**
    - * This class contains the mapping of file chunks to buffers inside BufferPool.
    - */
    -public class ChunkPool<K> /*implements EvictionListener*/ {
    - private final ConcurrentHashMap<K, Chunk> chunkCache = new ConcurrentHashMap<K, Chunk>();
    -
    - /** Number of unprocessed evictions, for the background thread. */
    - private final AtomicInteger newEvictions = new AtomicInteger(0);
    - private final Thread cleanupThread;
    -
    - public ChunkPool() {
    - cleanupThread = new CleanupThread();
    - cleanupThread.start();
    - }
    -
    - /**
    - * Gets a chunk from cache
    - * TODO: We expect that in most cases, some related chunks (e.g. columns for a stripe)
    - * will be stored in the same buffer. We could use this to get keys more efficiently.
    - * On the other hand, real stripes are pretty big.
    - * @param key key to search for.
    - * @return Chunk corresponding to k.
    - */
    - public Chunk getChunk(K key, HashSet<WeakBuffer> lockedBuffers) {
    - while (true) {
    - Chunk result = chunkCache.get(key);
    - if (result == null) return null;
    - if (lockChunk(result, lockedBuffers)) return result;
    - if (chunkCache.remove(key, result)) return null;
    - }
    - }
    -
    - private boolean lockChunk(Chunk result, HashSet<WeakBuffer> lockedBuffers) {
    - // We expect the chain to have 1 or 2 buffers (2 if we are on buffer boundary). Keep track of
    - // what we lock in the bitmask; may need fixing (extremely unlikely - 64+ buffer, giant chunks)
    - boolean failedToLock = false;
    - long blocksToUnlock = 0;
    - long bit = 1 << 63; // The bit indicating that current chunk was locked.
    -
    - Chunk chunk = result;
    - while (chunk != null) {
    - if (lockedBuffers.contains(chunk.buffer)) {
    - assert chunk.buffer.isLocked() : chunk.buffer + " is in lockedBuffers but is not locked";
    - } else if (chunk.buffer.lock(true)) {
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Locked " + chunk.buffer + " for " + result);
    - }
    - lockedBuffers.add(chunk.buffer);
    - blocksToUnlock += bit;
    - } else {
    - failedToLock = true;
    - break;
    - }
    - bit >>>= 1;
    - chunk = chunk.nextChunk;
    - if (bit == 1 && chunk != null) {
    - throw new AssertionError("Chunk chain was too long");
    - }
    - }
    - if (!failedToLock) return true;
    -
    - bit = 1 << 63;
    - Chunk chunk2 = result;
    - while (chunk2 != chunk) {
    - if ((blocksToUnlock & bit) == bit) {
    - if (DebugUtils.isTraceLockingEnabled()) {
    - LlapIoImpl.LOG.info("Unlocking " + chunk2.buffer + " due to failed chunk lock");
    - }
    - lockedBuffers.remove(chunk2.buffer);
    - chunk2.buffer.unlock();
    - }
    - bit >>>= 1;
    - chunk2 = chunk2.nextChunk;
    - }
    - return false;
    - }
    -
    - private boolean verifyChunk(Chunk entry) {
    - Chunk chunk = entry;
    - while (chunk != null) {
    - if (!chunk.buffer.lock(false)) break;
    - chunk = chunk.nextChunk;
    - }
    - Chunk chunk2 = entry;
    - while (chunk2 != chunk) {
    - chunk2.buffer.unlock();
    - chunk2 = chunk2.nextChunk;
    - }
    - return chunk == null;
    - }
    -
    - public Chunk addOrGetChunk(K key, Chunk val, HashSet<WeakBuffer> lockedBuffers) {
    - assert val.buffer.isLocked();
    - while (true) {
    - Chunk oldVal = chunkCache.putIfAbsent(key, val);
    - if (oldVal == null) return val;
    - if (DebugUtils.isTraceCachingEnabled()) {
    - LlapIoImpl.LOG.info("Trying to cache when the chunk is already cached for "
    - + key + "; old " + oldVal + ", new " + val);
    - }
    - if (lockChunk(oldVal, lockedBuffers)) return oldVal;
    - // We found some old value but couldn't lock it; remove it.
    - chunkCache.remove(key, oldVal);
    - }
    - }
    -
    - //@Override
    - public void evictionNotice(WeakBuffer evicted) {
    - int oldValue = newEvictions.getAndIncrement();
    - if (oldValue == 0) {
    - synchronized (newEvictions) {
    - newEvictions.notifyAll();
    - }
    - }
    - }
    -
    - public static class Chunk {
    - public WeakBuffer buffer;
    - public int offset, length;
    - public Chunk nextChunk;
    -
    - public Chunk(WeakBuffer buffer, int offset, int length) {
    - this.buffer = buffer;
    - this.offset = offset;
    - this.length = length;
    - }
    -
    - public Chunk addChunk(Chunk another) {
    - // Traversing list is bad; however, we expect that this will very rarely happen; and in
    - // nearly all the cases when it does (buffer boundary) the list will have 1 element.
    - Chunk chunk = this;
    - while (chunk.nextChunk != null) {
    - chunk = chunk.nextChunk;
    - }
    - chunk.nextChunk = another;
    - return this;
    - }
    -
    - @Override
    - public String toString() {
    - return "{" + buffer + ", " + offset + ", " + length + "}";
    - }
    -
    - public String toFullString() {
    - String result = "";
    - Chunk chunk = this;
    - while (chunk != null) {
    - result += chunk.toString() + ", ";
    - chunk = chunk.nextChunk;
    - }
    - return result;
    - }
    - }
    -
    - private final class CleanupThread extends Thread {
    - private int APPROX_CLEANUP_INTERVAL_SEC = 600;
    -
    - public CleanupThread() {
    - super("Llap ChunkPool cleanup thread");
    - setDaemon(true);
    - setPriority(1);
    - }
    -
    - @Override
    - public void run() {
    - while (true) {
    - try {
    - doOneCleanupRound();
    - } catch (InterruptedException ex) {
    - LlapIoImpl.LOG.warn("Cleanup thread has been interrupted");
    - Thread.currentThread().interrupt();
    - break;
    - } catch (Throwable t) {
    - LlapIoImpl.LOG.error("Cleanup has failed; the thread will now exit", t);
    - break;
    - }
    - }
    - }
    -
    - private void doOneCleanupRound() throws InterruptedException {
    - while (true) {
    - int evictionsSinceLast = newEvictions.getAndSet(0);
    - if (evictionsSinceLast > 0) break;
    - synchronized (newEvictions) {
    - newEvictions.wait(10000);
    - }
    - }
    - // Duration is an estimate; if the size of the map changes rapidly, it can be very different.
    - long endTime = System.nanoTime() + APPROX_CLEANUP_INTERVAL_SEC * 1000000000L;
    - int processed = 0;
    - // TODO: if this iterator affects the map in some bad way,
    - // we'd need to sleep once per round instead.
    - Iterator<Map.Entry<K, Chunk>> iter = chunkCache.entrySet().iterator();
    - while (iter.hasNext()) {
    - if (!verifyChunk(iter.next().getValue())) {
    - iter.remove();
    - }
    - ++processed;
    - int approxElementsLeft = chunkCache.size() - processed;
    - Thread.sleep((approxElementsLeft <= 0)
    - ? 1 : (endTime - System.nanoTime()) / (1000000L * approxElementsLeft));
    - }
    - }
    - }
    -}

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/llap-server/src/main/resources/llap-daemon-log4j2.properties
    ----------------------------------------------------------------------
    diff --git a/llap-server/src/main/resources/llap-daemon-log4j2.properties b/llap-server/src/main/resources/llap-daemon-log4j2.properties
    index 5051ca5..268eb59 100644
    --- a/llap-server/src/main/resources/llap-daemon-log4j2.properties
    +++ b/llap-server/src/main/resources/llap-daemon-log4j2.properties
    @@ -64,7 +64,19 @@ appender.HISTORYAPPENDER.strategy.type = DefaultRolloverStrategy
      appender.HISTORYAPPENDER.strategy.max = ${sys:llap.daemon.log.maxbackupindex}

      # list of all loggers
    -loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger
    +loggers = NIOServerCnxn, ClientCnxnSocketNIO, DataNucleus, Datastore, JPOX, HistoryLogger, LlapIoImpl, LlapIoOrc, LlapIoCache, LlapIoLocking
    +
    +logger.LlapIoImpl.name = LlapIoImpl
    +logger.LlapIoImpl.level = INFO
    +
    +logger.LlapIoOrc.name = LlapIoOrc
    +logger.LlapIoOrc.level = WARN
    +
    +logger.LlapIoCache.name = LlapIoCache
    +logger.LlapIOCache.level = WARN
    +
    +logger.LlapIoLocking.name = LlapIoLocking
    +logger.LlapIoLocking.level = WARN

      logger.NIOServerCnxn.name = org.apache.zookeeper.server.NIOServerCnxn
      logger.NIOServerCnxn.level = WARN

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/ql/src/java/org/apache/hadoop/hive/llap/DebugUtils.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/llap/DebugUtils.java b/ql/src/java/org/apache/hadoop/hive/llap/DebugUtils.java
    index ea626d7..3d81e43 100644
    --- a/ql/src/java/org/apache/hadoop/hive/llap/DebugUtils.java
    +++ b/ql/src/java/org/apache/hadoop/hive/llap/DebugUtils.java
    @@ -23,49 +23,6 @@ package org.apache.hadoop.hive.llap;
       * trace messages with low runtime cost, in order to investigate reproducible bugs.
       */
      public class DebugUtils {
    -
    - public static boolean isTraceEnabled() {
    - return false;
    - }
    -
    - public static boolean isTraceOrcEnabled() {
    - return false;
    - }
    -
    - public static boolean isTraceLockingEnabled() {
    - return false;
    - }
    -
    - public static boolean isTraceMttEnabled() {
    - return false;
    - }
    -
    - public static boolean isTraceCachingEnabled() {
    - return false;
    - }
    -
    - public static String toString(long[] a, int offset, int len) {
    - StringBuilder b = new StringBuilder();
    - b.append('[');
    - for (int i = offset; i < offset + len; ++i) {
    - b.append(a[i]);
    - b.append(", ");
    - }
    - b.append(']');
    - return b.toString();
    - }
    -
    - public static String toString(byte[] a, int offset, int len) {
    - StringBuilder b = new StringBuilder();
    - b.append('[');
    - for (int i = offset; i < offset + len; ++i) {
    - b.append(a[i]);
    - b.append(", ");
    - }
    - b.append(']');
    - return b.toString();
    - }
    -
        public static String toString(boolean[] a) {
          StringBuilder b = new StringBuilder();
          b.append('[');

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java b/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java
    deleted file mode 100644
    index 300230f..0000000
    --- a/ql/src/java/org/apache/hadoop/hive/llap/LogLevels.java
    +++ /dev/null
    @@ -1,53 +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 debugrmation
    - * 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.hadoop.hive.llap;
    -
    -import org.slf4j.Logger;
    -
    -public class LogLevels {
    - private final boolean isT, isD, isI, isW, isE;
    -
    - public LogLevels(Logger log) {
    - isT = log.isTraceEnabled();
    - isD = log.isDebugEnabled();
    - isI = log.isInfoEnabled();
    - isW = log.isWarnEnabled();
    - isE = log.isErrorEnabled();
    - }
    -
    - public boolean isTraceEnabled() {
    - return isT;
    - }
    -
    - public boolean isDebugEnabled() {
    - return isD;
    - }
    -
    - public boolean isInfoEnabled() {
    - return isI;
    - }
    -
    - public boolean isWarnEnabled() {
    - return isW;
    - }
    -
    - public boolean isErrorEnabled() {
    - return isE;
    - }
    -}

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
    index b8490df..31f5c72 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
    @@ -815,7 +815,7 @@ public class VectorGroupByOperator extends Operator<GroupByDesc> implements
              aggregationBatchInfo = new VectorAggregationBufferBatch();
              aggregationBatchInfo.compileAggregationBatchInfo(aggregators);
            }
    - LOG.warn("VectorGroupByOperator is vector output " + isVectorOutput);
    + LOG.info("VectorGroupByOperator is vector output {}", isVectorOutput);
            outputObjInspector = ObjectInspectorFactory.getStandardStructObjectInspector(
                outputFieldNames, objectInspectors);
            if (isVectorOutput) {

    http://git-wip-us.apache.org/repos/asf/hive/blob/3b6b56d7/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
    index 96af96a..4d09dcd 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/encoded/EncodedReader.java
    @@ -54,5 +54,5 @@ public interface EncodedReader {
         * checks are entirely eliminated because this method is called with constant value, similar
         * to just checking the constant in the first place.
         */
    - void setDebugTracing(boolean isEnabled);
    + void setTracing(boolean isEnabled);
      }
    \ No newline at end of file
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13324. LLAP: history log for FRAGMENT_START doesn't log DagId correctly. (Siddharth Seth, Reviewed by Sergey Shelukhin)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3038b05e
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3038b05e
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3038b05e

    Branch: refs/heads/llap
    Commit: 3038b05ed346f4b5438e9072edb19186ea90d042
    Parents: 2449d1d
    Author: Siddharth Seth <sseth@apache.org>
    Authored: Sat Mar 26 14:12:36 2016 -0700
    Committer: Siddharth Seth <sseth@apache.org>
    Committed: Sat Mar 26 14:12:36 2016 -0700

    ----------------------------------------------------------------------
      .../apache/hadoop/hive/llap/tez/Converters.java | 1 +
      .../hadoop/hive/llap/tez/TestConverters.java | 190 +++++++++++++++++++
      2 files changed, 191 insertions(+)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/3038b05e/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
    ----------------------------------------------------------------------
    diff --git a/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java b/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
    index a5c3631..ec6e439 100644
    --- a/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
    +++ b/llap-common/src/java/org/apache/hadoop/hive/llap/tez/Converters.java
    @@ -85,6 +85,7 @@ public class Converters {
          FragmentSpecProto.Builder builder = FragmentSpecProto.newBuilder();
          builder.setFragmentIdentifierString(taskSpec.getTaskAttemptID().toString());
          builder.setDagName(taskSpec.getDAGName());
    + builder.setDagId(taskSpec.getDagIdentifier());
          builder.setVertexName(taskSpec.getVertexName());
          builder.setVertexParallelism(taskSpec.getVertexParallelism());
          builder.setFragmentNumber(taskSpec.getTaskAttemptID().getTaskID().getId());

    http://git-wip-us.apache.org/repos/asf/hive/blob/3038b05e/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
    ----------------------------------------------------------------------
    diff --git a/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java b/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
    new file mode 100644
    index 0000000..d4cdac1
    --- /dev/null
    +++ b/llap-common/src/test/org/apache/hadoop/hive/llap/tez/TestConverters.java
    @@ -0,0 +1,190 @@
    +/*
    + * Licensed 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.hadoop.hive.llap.tez;
    +
    +import static org.junit.Assert.assertArrayEquals;
    +import static org.junit.Assert.assertEquals;
    +
    +import java.nio.ByteBuffer;
    +import java.util.List;
    +
    +import com.google.common.collect.Lists;
    +import com.google.protobuf.ByteString;
    +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.EntityDescriptorProto;
    +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.FragmentSpecProto;
    +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.IOSpecProto;
    +import org.apache.hadoop.hive.llap.daemon.rpc.LlapDaemonProtocolProtos.UserPayloadProto;
    +import org.apache.hadoop.yarn.api.records.ApplicationId;
    +import org.apache.tez.dag.api.InputDescriptor;
    +import org.apache.tez.dag.api.OutputDescriptor;
    +import org.apache.tez.dag.api.ProcessorDescriptor;
    +import org.apache.tez.dag.api.UserPayload;
    +import org.apache.tez.dag.records.TezDAGID;
    +import org.apache.tez.dag.records.TezTaskAttemptID;
    +import org.apache.tez.dag.records.TezTaskID;
    +import org.apache.tez.dag.records.TezVertexID;
    +import org.apache.tez.runtime.api.impl.InputSpec;
    +import org.apache.tez.runtime.api.impl.OutputSpec;
    +import org.apache.tez.runtime.api.impl.TaskSpec;
    +import org.junit.Test;
    +
    +public class TestConverters {
    +
    + @Test(timeout = 5000)
    + public void testTaskSpecToFragmentSpec() {
    + ByteBuffer procBb = ByteBuffer.allocate(4);
    + procBb.putInt(0, 200);
    + UserPayload processorPayload = UserPayload.create(procBb);
    + ProcessorDescriptor processorDescriptor =
    + ProcessorDescriptor.create("fakeProcessorName").setUserPayload(processorPayload);
    +
    + ByteBuffer input1Bb = ByteBuffer.allocate(4);
    + input1Bb.putInt(0, 300);
    + UserPayload input1Payload = UserPayload.create(input1Bb);
    + InputDescriptor id1 = InputDescriptor.create("input1ClassName").setUserPayload(input1Payload);
    + InputSpec inputSpec1 = new InputSpec("sourceVertexName1", id1, 33);
    + InputSpec inputSpec2 = new InputSpec("sourceVertexName2", id1, 44);
    + List<InputSpec> inputSpecList = Lists.newArrayList(inputSpec1, inputSpec2);
    +
    + ByteBuffer output1Bb = ByteBuffer.allocate(4);
    + output1Bb.putInt(0, 400);
    + UserPayload output1Payload = UserPayload.create(output1Bb);
    + OutputDescriptor od1 =
    + OutputDescriptor.create("output1ClassName").setUserPayload(output1Payload);
    + OutputSpec outputSpec1 = new OutputSpec("destVertexName1", od1, 55);
    + OutputSpec outputSpec2 = new OutputSpec("destVertexName2", od1, 66);
    + List<OutputSpec> outputSpecList = Lists.newArrayList(outputSpec1, outputSpec2);
    +
    + ApplicationId appId = ApplicationId.newInstance(1000, 100);
    + TezDAGID tezDagId = TezDAGID.getInstance(appId, 300);
    + TezVertexID tezVertexId = TezVertexID.getInstance(tezDagId, 400);
    + TezTaskID tezTaskId = TezTaskID.getInstance(tezVertexId, 500);
    + TezTaskAttemptID tezTaskAttemptId = TezTaskAttemptID.getInstance(tezTaskId, 600);
    +
    + TaskSpec taskSpec =
    + new TaskSpec(tezTaskAttemptId, "dagName", "vertexName", 10, processorDescriptor,
    + inputSpecList, outputSpecList, null);
    +
    +
    + FragmentSpecProto fragmentSpecProto = Converters.convertTaskSpecToProto(taskSpec);
    +
    +
    + assertEquals("dagName", fragmentSpecProto.getDagName());
    + assertEquals("vertexName", fragmentSpecProto.getVertexName());
    + assertEquals(tezTaskAttemptId.toString(), fragmentSpecProto.getFragmentIdentifierString());
    + assertEquals(tezDagId.getId(), fragmentSpecProto.getDagId());
    + assertEquals(tezTaskAttemptId.getId(), fragmentSpecProto.getAttemptNumber());
    + assertEquals(tezTaskId.getId(), fragmentSpecProto.getFragmentNumber());
    + assertEquals(processorDescriptor.getClassName(),
    + fragmentSpecProto.getProcessorDescriptor().getClassName());
    + assertEquals(processorDescriptor.getUserPayload().getPayload(),
    + fragmentSpecProto.getProcessorDescriptor().getUserPayload().getUserPayload()
    + .asReadOnlyByteBuffer());
    + assertEquals(2, fragmentSpecProto.getInputSpecsCount());
    + assertEquals(2, fragmentSpecProto.getOutputSpecsCount());
    +
    + verifyInputSpecAndProto(inputSpec1, fragmentSpecProto.getInputSpecs(0));
    + verifyInputSpecAndProto(inputSpec2, fragmentSpecProto.getInputSpecs(1));
    + verifyOutputSpecAndProto(outputSpec1, fragmentSpecProto.getOutputSpecs(0));
    + verifyOutputSpecAndProto(outputSpec2, fragmentSpecProto.getOutputSpecs(1));
    +
    + }
    +
    + @Test (timeout = 5000)
    + public void testFragmentSpecToTaskSpec() {
    +
    + ByteBuffer procBb = ByteBuffer.allocate(4);
    + procBb.putInt(0, 200);
    +
    + ByteBuffer input1Bb = ByteBuffer.allocate(4);
    + input1Bb.putInt(0, 300);
    +
    + ByteBuffer output1Bb = ByteBuffer.allocate(4);
    + output1Bb.putInt(0, 400);
    +
    + ApplicationId appId = ApplicationId.newInstance(1000, 100);
    + TezDAGID tezDagId = TezDAGID.getInstance(appId, 300);
    + TezVertexID tezVertexId = TezVertexID.getInstance(tezDagId, 400);
    + TezTaskID tezTaskId = TezTaskID.getInstance(tezVertexId, 500);
    + TezTaskAttemptID tezTaskAttemptId = TezTaskAttemptID.getInstance(tezTaskId, 600);
    +
    + FragmentSpecProto.Builder builder = FragmentSpecProto.newBuilder();
    + builder.setFragmentIdentifierString(tezTaskAttemptId.toString());
    + builder.setDagName("dagName");
    + builder.setVertexName("vertexName");
    + builder.setDagId(tezDagId.getId());
    + builder.setProcessorDescriptor(
    + EntityDescriptorProto.newBuilder().setClassName("fakeProcessorName").setUserPayload(
    + UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(procBb))));
    + builder.addInputSpecs(IOSpecProto.newBuilder().setConnectedVertexName("sourceVertexName1")
    + .setPhysicalEdgeCount(33).setIoDescriptor(
    + EntityDescriptorProto.newBuilder().setClassName("input1ClassName").setUserPayload(
    + UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(input1Bb)))));
    + builder.addInputSpecs(IOSpecProto.newBuilder().setConnectedVertexName("sourceVertexName2")
    + .setPhysicalEdgeCount(44).setIoDescriptor(
    + EntityDescriptorProto.newBuilder().setClassName("input1ClassName").setUserPayload(
    + UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(input1Bb)))));
    + builder.addOutputSpecs(IOSpecProto.newBuilder().setConnectedVertexName("destVertexName1")
    + .setPhysicalEdgeCount(55).setIoDescriptor(
    + EntityDescriptorProto.newBuilder().setClassName("outputClassName").setUserPayload(
    + UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(output1Bb)))));
    + builder.addOutputSpecs(IOSpecProto.newBuilder().setConnectedVertexName("destVertexName2")
    + .setPhysicalEdgeCount(66).setIoDescriptor(
    + EntityDescriptorProto.newBuilder().setClassName("outputClassName").setUserPayload(
    + UserPayloadProto.newBuilder().setUserPayload(ByteString.copyFrom(output1Bb)))));
    +
    + FragmentSpecProto fragmentSpecProto = builder.build();
    +
    + TaskSpec taskSpec = Converters.getTaskSpecfromProto(fragmentSpecProto);
    +
    + assertEquals("dagName", taskSpec.getDAGName());
    + assertEquals("vertexName", taskSpec.getVertexName());
    + assertEquals(tezTaskAttemptId, taskSpec.getTaskAttemptID());
    + assertEquals("fakeProcessorName", taskSpec.getProcessorDescriptor().getClassName());
    + byte[] serialized = new byte[taskSpec.getProcessorDescriptor().getUserPayload().getPayload().remaining()];
    + taskSpec.getProcessorDescriptor().getUserPayload().getPayload().get(serialized);
    + assertArrayEquals(procBb.array(), serialized);
    +
    + assertEquals(2, taskSpec.getInputs().size());
    + assertEquals(2, taskSpec.getOutputs().size());
    +
    + verifyInputSpecAndProto(taskSpec.getInputs().get(0), fragmentSpecProto.getInputSpecs(0));
    + verifyInputSpecAndProto(taskSpec.getInputs().get(1), fragmentSpecProto.getInputSpecs(1));
    + verifyOutputSpecAndProto(taskSpec.getOutputs().get(0), fragmentSpecProto.getOutputSpecs(0));
    + verifyOutputSpecAndProto(taskSpec.getOutputs().get(1), fragmentSpecProto.getOutputSpecs(1));
    +
    +
    + }
    +
    + private void verifyInputSpecAndProto(InputSpec inputSpec,
    + IOSpecProto inputSpecProto) {
    + assertEquals(inputSpec.getPhysicalEdgeCount(), inputSpecProto.getPhysicalEdgeCount());
    + assertEquals(inputSpec.getSourceVertexName(), inputSpecProto.getConnectedVertexName());
    + assertEquals(inputSpec.getInputDescriptor().getClassName(),
    + inputSpecProto.getIoDescriptor().getClassName());
    + assertEquals(inputSpec.getInputDescriptor().getUserPayload().getPayload(),
    + inputSpecProto.getIoDescriptor().getUserPayload().getUserPayload().asReadOnlyByteBuffer());
    + }
    +
    + private void verifyOutputSpecAndProto(OutputSpec outputSpec,
    + IOSpecProto outputSpecProto) {
    + assertEquals(outputSpec.getPhysicalEdgeCount(), outputSpecProto.getPhysicalEdgeCount());
    + assertEquals(outputSpec.getDestinationVertexName(), outputSpecProto.getConnectedVertexName());
    + assertEquals(outputSpec.getOutputDescriptor().getClassName(),
    + outputSpecProto.getIoDescriptor().getClassName());
    + assertEquals(outputSpec.getOutputDescriptor().getUserPayload().getPayload(),
    + outputSpecProto.getIoDescriptor().getUserPayload().getUserPayload().asReadOnlyByteBuffer());
    + }
    +}
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13151 : Clean up UGI objects in FileSystem cache for transactions, ADDENDUM (Wei Zheng, reviewed by Eugene Koifman)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4fabd038
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4fabd038
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4fabd038

    Branch: refs/heads/llap
    Commit: 4fabd038cf64b906a89726805958c43b97194291
    Parents: 6bfec2e
    Author: Wei Zheng <weiz@apache.org>
    Authored: Thu Mar 24 22:18:32 2016 -0700
    Committer: Wei Zheng <weiz@apache.org>
    Committed: Thu Mar 24 22:18:32 2016 -0700

    ----------------------------------------------------------------------
      .../java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java | 5 +++--
      .../java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java | 4 ++--
      ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java | 5 +++--
      3 files changed, 8 insertions(+), 6 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/4fabd038/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    index 4c31a49..23b1b7f 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Cleaner.java
    @@ -275,8 +275,9 @@ public class Cleaner extends CompactorThread {
              try {
                FileSystem.closeAllForUGI(ugi);
              } catch (IOException exception) {
    - LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception + " for " +
    - ci.getFullPartitionName()); }
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
    + ci.getFullPartitionName(), exception);
    + }
            }
            txnHandler.markCleaned(ci);
          } catch (Exception e) {

    http://git-wip-us.apache.org/repos/asf/hive/blob/4fabd038/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    index 98ebf53..abbe5d4 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    @@ -235,8 +235,8 @@ public class Initiator extends CompactorThread {
            try {
              FileSystem.closeAllForUGI(ugi);
            } catch (IOException exception) {
    - LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception + " for " +
    - ci.getFullPartitionName());
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
    + ci.getFullPartitionName(), exception);
            }
            return compactionType;
          }

    http://git-wip-us.apache.org/repos/asf/hive/blob/4fabd038/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    index e21ca27..6238e2b 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
    @@ -175,8 +175,9 @@ public class Worker extends CompactorThread {
                  try {
                    FileSystem.closeAllForUGI(ugi);
                  } catch (IOException exception) {
    - LOG.error("Could not clean up file-system handles for UGI: " + ugi, exception + " for " +
    - ci.getFullPartitionName()); }
    + LOG.error("Could not clean up file-system handles for UGI: " + ugi + " for " +
    + ci.getFullPartitionName(), exception);
    + }
                }
                txnHandler.markCompacted(ci);
              } catch (Exception e) {
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13115: MetaStore Direct SQL getPartitions call fail when the columns schemas for a partition are null (Ratandeep Ratti reviewed by Carl Steinbach)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/69cfd357
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/69cfd357
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/69cfd357

    Branch: refs/heads/llap
    Commit: 69cfd357eb482c426161aa3c4a00d574ee29416e
    Parents: 9686209
    Author: Carl Steinbach <cws@apache.org>
    Authored: Sun Mar 27 15:41:38 2016 -0700
    Committer: Carl Steinbach <cws@apache.org>
    Committed: Sun Mar 27 15:41:38 2016 -0700

    ----------------------------------------------------------------------
      .../hive/metastore/TestHiveMetaStore.java | 20 +++++++++++-
      .../hive/metastore/MetaStoreDirectSql.java | 32 +++++++++++---------
      2 files changed, 37 insertions(+), 15 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/69cfd357/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
    ----------------------------------------------------------------------
    diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
    index 5da4165..83fb15c 100644
    --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
    +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStore.java
    @@ -280,6 +280,24 @@ public abstract class TestHiveMetaStore extends TestCase {
            }
            assertTrue("Partitions are not same", part.equals(part_get));

    + // check null cols schemas for a partition
    + List<String> vals6 = makeVals("2016-02-22 00:00:00", "16");
    + Partition part6 = makePartitionObject(dbName, tblName, vals6, tbl, "/part5");
    + part6.getSd().setCols(null);
    + LOG.info("Creating partition will null field schema");
    + client.add_partition(part6);
    + LOG.info("Listing all partitions for table " + dbName + "." + tblName);
    + final List<Partition> partitions = client.listPartitions(dbName, tblName, (short) -1);
    + boolean foundPart = false;
    + for (Partition p : partitions) {
    + if (p.getValues().equals(vals6)) {
    + assertNull(p.getSd().getCols());
    + LOG.info("Found partition " + p + " having null field schema");
    + foundPart = true;
    + }
    + }
    + assertTrue(foundPart);
    +
            String partName = "ds=" + FileUtils.escapePathName("2008-07-01 14:13:12") + "/hr=14";
            String part2Name = "ds=" + FileUtils.escapePathName("2008-07-01 14:13:12") + "/hr=15";
            String part3Name = "ds=" + FileUtils.escapePathName("2008-07-02 14:13:12") + "/hr=15";
    @@ -313,7 +331,7 @@ public abstract class TestHiveMetaStore extends TestCase {
            partialVals.clear();
            partialVals.add("");
            partialNames = client.listPartitionNames(dbName, tblName, partialVals, (short) -1);
    - assertTrue("Should have returned 4 partition names", partialNames.size() == 4);
    + assertTrue("Should have returned 5 partition names", partialNames.size() == 5);
            assertTrue("Not all part names returned", partialNames.containsAll(partNames));

            // Test partition listing with a partial spec - hr is specified but ds is not

    http://git-wip-us.apache.org/repos/asf/hive/blob/69cfd357/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
    index d51f58d..06e9f78 100644
    --- a/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
    +++ b/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java
    @@ -557,14 +557,14 @@ class MetaStoreDirectSql {
            Long sdId = extractSqlLong(fields[1]);
            Long colId = extractSqlLong(fields[2]);
            Long serdeId = extractSqlLong(fields[3]);
    - // A partition must have either everything set, or nothing set if it's a view.
    - if (sdId == null || colId == null || serdeId == null) {
    + // A partition must have at least sdId and serdeId set, or nothing set if it's a view.
    + if (sdId == null || serdeId == null) {
              if (isView == null) {
                isView = isViewTable(dbName, tblName);
              }
              if ((sdId != null || colId != null || serdeId != null) || !isView) {
    - throw new MetaException("Unexpected null for one of the IDs, SD " + sdId + ", column "
    - + colId + ", serde " + serdeId + " for a " + (isView ? "" : "non-") + " view");
    + throw new MetaException("Unexpected null for one of the IDs, SD " + sdId +
    + ", serde " + serdeId + " for a " + (isView ? "" : "non-") + " view");
              }
            }

    @@ -580,7 +580,7 @@ class MetaStoreDirectSql {
            partitions.put(partitionId, part);

            if (sdId == null) continue; // Probably a view.
    - assert colId != null && serdeId != null;
    + assert serdeId != null;

            // We assume each partition has an unique SD.
            StorageDescriptor sd = new StorageDescriptor();
    @@ -605,14 +605,16 @@ class MetaStoreDirectSql {
            sdSb.append(sdId).append(",");
            part.setSd(sd);

    - List<FieldSchema> cols = colss.get(colId);
    - // We expect that colId will be the same for all (or many) SDs.
    - if (cols == null) {
    - cols = new ArrayList<FieldSchema>();
    - colss.put(colId, cols);
    - colsSb.append(colId).append(",");
    + if (colId != null) {
    + List<FieldSchema> cols = colss.get(colId);
    + // We expect that colId will be the same for all (or many) SDs.
    + if (cols == null) {
    + cols = new ArrayList<FieldSchema>();
    + colss.put(colId, cols);
    + colsSb.append(colId).append(",");
    + }
    + sd.setCols(cols);
            }
    - sd.setCols(cols);

            // We assume each SD has an unique serde.
            SerDeInfo serde = new SerDeInfo();
    @@ -658,8 +660,10 @@ class MetaStoreDirectSql {
            assert serdeSb.length() == 0 && colsSb.length() == 0;
            return orderedResult; // No SDs, probably a view.
          }
    - String sdIds = trimCommaList(sdSb), serdeIds = trimCommaList(serdeSb),
    - colIds = trimCommaList(colsSb);
    +
    + String sdIds = trimCommaList(sdSb);
    + String serdeIds = trimCommaList(serdeSb);
    + String colIds = trimCommaList(colsSb);

          // Get all the stuff for SD. Don't do empty-list check - we expect partitions do have SDs.
          queryText = "select \"SD_ID\", \"PARAM_KEY\", \"PARAM_VALUE\" from \"SD_PARAMS\""
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12653 : The property "serialization.encoding" in the class "org.apache.hadoop.hive.contrib.serde2.MultiDelimitSerDe" does not work (yangfeng via Ashutosh Chauhan)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2449d1df
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2449d1df
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2449d1df

    Branch: refs/heads/llap
    Commit: 2449d1dfe9429363a9458d2004ec2405f5aa9035
    Parents: e384b2b
    Author: yangfang <yang.fang2@zte.com.cn>
    Authored: Mon Dec 14 03:13:00 2015 -0800
    Committer: Ashutosh Chauhan <hashutosh@apache.org>
    Committed: Fri Mar 25 07:43:05 2016 -0700

    ----------------------------------------------------------------------
      .../hive/contrib/serde2/MultiDelimitSerDe.java | 23 +++++++++++++++-----
      1 file changed, 18 insertions(+), 5 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/2449d1df/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/MultiDelimitSerDe.java
    ----------------------------------------------------------------------
    diff --git a/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/MultiDelimitSerDe.java b/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/MultiDelimitSerDe.java
    index 9a162d5..296c449 100644
    --- a/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/MultiDelimitSerDe.java
    +++ b/contrib/src/java/org/apache/hadoop/hive/contrib/serde2/MultiDelimitSerDe.java
    @@ -63,7 +63,7 @@ import org.apache.hadoop.io.Writable;
          serdeConstants.SERIALIZATION_ENCODING,
          LazySerDeParameters.SERIALIZATION_EXTEND_NESTING_LEVELS,
          LazySerDeParameters.SERIALIZATION_EXTEND_ADDITIONAL_NESTING_LEVELS})
    -public class MultiDelimitSerDe extends AbstractSerDe {
    +public class MultiDelimitSerDe extends AbstractEncodingAwareSerDe {

        private static final byte[] DEFAULT_SEPARATORS = {(byte) 1, (byte) 2, (byte) 3};
        // Due to HIVE-6404, define our own constant
    @@ -94,6 +94,7 @@ public class MultiDelimitSerDe extends AbstractSerDe {
        @Override
        public void initialize(Configuration conf, Properties tbl) throws SerDeException {
          // get the SerDe parameters
    + super.initialize(conf, tbl);
          serdeParams = new LazySerDeParameters(conf, tbl, getClass().getName());

          fieldDelimited = tbl.getProperty(serdeConstants.FIELD_DELIM);
    @@ -134,8 +135,9 @@ public class MultiDelimitSerDe extends AbstractSerDe {
          return Text.class;
        }

    - @Override
    - public Object deserialize(Writable blob) throws SerDeException {
    +
    + @Override
    + public Object doDeserialize(Writable blob) throws SerDeException {
          if (byteArrayRef == null) {
            byteArrayRef = new ByteArrayRef();
          }
    @@ -159,8 +161,9 @@ public class MultiDelimitSerDe extends AbstractSerDe {
          return cachedLazyStruct;
        }

    - @Override
    - public Writable serialize(Object obj, ObjectInspector objInspector) throws SerDeException {
    + @Override
    + public Writable doSerialize(Object obj, ObjectInspector objInspector)
    + throws SerDeException {
          StructObjectInspector soi = (StructObjectInspector) objInspector;
          List<? extends StructField> fields = soi.getAllStructFieldRefs();
          List<Object> list = soi.getStructFieldsDataAsList(obj);
    @@ -286,6 +289,16 @@ public class MultiDelimitSerDe extends AbstractSerDe {
          throw new RuntimeException("Unknown category type: "+ objInspector.getCategory());
        }

    + protected Text transformFromUTF8(Writable blob) {
    + Text text = (Text)blob;
    + return SerDeUtils.transformTextFromUTF8(text, this.charset);
    + }
    +
    + protected Text transformToUTF8(Writable blob) {
    + Text text = (Text) blob;
    + return SerDeUtils.transformTextToUTF8(text, this.charset);
    + }
    +
        @Override
        public SerDeStats getSerDeStats() {
          // no support for statistics
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out b/ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out
    new file mode 100644
    index 0000000..8409a01
    --- /dev/null
    +++ b/ql/src/test/results/clientpositive/tez/vector_interval_arithmetic.q.out
    @@ -0,0 +1,1086 @@
    +PREHOOK: query: create table unique_timestamps (tsval timestamp) STORED AS TEXTFILE
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@unique_timestamps
    +POSTHOOK: query: create table unique_timestamps (tsval timestamp) STORED AS TEXTFILE
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@unique_timestamps
    +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/timestamps.txt' OVERWRITE INTO TABLE unique_timestamps
    +PREHOOK: type: LOAD
    +#### A masked pattern was here ####
    +PREHOOK: Output: default@unique_timestamps
    +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/timestamps.txt' OVERWRITE INTO TABLE unique_timestamps
    +POSTHOOK: type: LOAD
    +#### A masked pattern was here ####
    +POSTHOOK: Output: default@unique_timestamps
    +PREHOOK: query: create table interval_arithmetic_1 (dateval date, tsval timestamp) stored as orc
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: query: create table interval_arithmetic_1 (dateval date, tsval timestamp) stored as orc
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@interval_arithmetic_1
    +PREHOOK: query: insert overwrite table interval_arithmetic_1
    + select cast(tsval as date), tsval from unique_timestamps
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@unique_timestamps
    +PREHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: query: insert overwrite table interval_arithmetic_1
    + select cast(tsval as date), tsval from unique_timestamps
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@unique_timestamps
    +POSTHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: Lineage: interval_arithmetic_1.dateval EXPRESSION [(unique_timestamps)unique_timestamps.FieldSchema(name:tsval, type:timestamp, comment:null), ]
    +POSTHOOK: Lineage: interval_arithmetic_1.tsval SIMPLE [(unique_timestamps)unique_timestamps.FieldSchema(name:tsval, type:timestamp, comment:null), ]
    +_c0 tsval
    +PREHOOK: query: -- interval year-month arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: -- interval year-month arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), (dateval - 2-2) (type: date), (dateval - -2-2) (type: date), (dateval + 2-2) (type: date), (dateval + -2-2) (type: date), (-2-2 + dateval) (type: date), (2-2 + dateval) (type: date)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: date), _col2 (type: date), _col3 (type: date), _col4 (type: date), _col5 (type: date), _col6 (type: date)
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: date), VALUE._col1 (type: date), VALUE._col2 (type: date), VALUE._col3 (type: date), VALUE._col4 (type: date), VALUE._col5 (type: date)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval c1 c2 c3 c4 c5 c6
    +0004-09-22 0002-07-22 0006-11-22 0006-11-22 0002-07-22 0002-07-22 0006-11-22
    +0528-10-27 0526-08-27 0530-12-27 0530-12-27 0526-08-27 0526-08-27 0530-12-27
    +1319-02-02 1316-12-02 1321-04-02 1321-04-02 1316-12-02 1316-12-02 1321-04-02
    +1404-07-23 1402-05-23 1406-09-23 1406-09-23 1402-05-23 1402-05-23 1406-09-23
    +1815-05-06 1813-03-06 1817-07-06 1817-07-06 1813-03-06 1813-03-06 1817-07-06
    +1883-04-17 1881-02-17 1885-06-17 1885-06-17 1881-02-17 1881-02-17 1885-06-17
    +1966-08-16 1964-06-16 1968-10-16 1968-10-16 1964-06-16 1964-06-16 1968-10-16
    +1973-04-17 1971-02-17 1975-06-17 1975-06-17 1971-02-17 1971-02-17 1975-06-17
    +1974-10-04 1972-08-04 1976-12-04 1976-12-04 1972-08-04 1972-08-04 1976-12-04
    +1976-03-03 1974-01-03 1978-05-03 1978-05-03 1974-01-03 1974-01-03 1978-05-03
    +1976-05-06 1974-03-06 1978-07-06 1978-07-06 1974-03-06 1974-03-06 1978-07-06
    +1978-08-05 1976-06-05 1980-10-05 1980-10-05 1976-06-05 1976-06-05 1980-10-05
    +1981-04-25 1979-02-25 1983-06-25 1983-06-25 1979-02-25 1979-02-25 1983-06-25
    +1981-11-15 1979-09-15 1984-01-15 1984-01-15 1979-09-15 1979-09-15 1984-01-15
    +1985-07-20 1983-05-20 1987-09-20 1987-09-20 1983-05-20 1983-05-20 1987-09-20
    +1985-11-18 1983-09-18 1988-01-18 1988-01-18 1983-09-18 1983-09-18 1988-01-18
    +1987-02-21 1984-12-21 1989-04-21 1989-04-21 1984-12-21 1984-12-21 1989-04-21
    +1987-05-28 1985-03-28 1989-07-28 1989-07-28 1985-03-28 1985-03-28 1989-07-28
    +1998-10-16 1996-08-16 2000-12-16 2000-12-16 1996-08-16 1996-08-16 2000-12-16
    +1999-10-03 1997-08-03 2001-12-03 2001-12-03 1997-08-03 1997-08-03 2001-12-03
    +2000-12-18 1998-10-18 2003-02-18 2003-02-18 1998-10-18 1998-10-18 2003-02-18
    +2002-05-10 2000-03-10 2004-07-10 2004-07-10 2000-03-10 2000-03-10 2004-07-10
    +2003-09-23 2001-07-23 2005-11-23 2005-11-23 2001-07-23 2001-07-23 2005-11-23
    +2004-03-07 2002-01-07 2006-05-07 2006-05-07 2002-01-07 2002-01-07 2006-05-07
    +2007-02-09 2004-12-09 2009-04-09 2009-04-09 2004-12-09 2004-12-09 2009-04-09
    +2009-01-21 2006-11-21 2011-03-21 2011-03-21 2006-11-21 2006-11-21 2011-03-21
    +2010-04-08 2008-02-08 2012-06-08 2012-06-08 2008-02-08 2008-02-08 2012-06-08
    +2013-04-07 2011-02-07 2015-06-07 2015-06-07 2011-02-07 2011-02-07 2015-06-07
    +2013-04-10 2011-02-10 2015-06-10 2015-06-10 2011-02-10 2011-02-10 2015-06-10
    +2021-09-24 2019-07-24 2023-11-24 2023-11-24 2019-07-24 2019-07-24 2023-11-24
    +2024-11-11 2022-09-11 2027-01-11 2027-01-11 2022-09-11 2022-09-11 2027-01-11
    +4143-07-08 4141-05-08 4145-09-08 4145-09-08 4141-05-08 4141-05-08 4145-09-08
    +4966-12-04 4964-10-04 4969-02-04 4969-02-04 4964-10-04 4964-10-04 4969-02-04
    +5339-02-01 5336-12-01 5341-04-01 5341-04-01 5336-12-01 5336-12-01 5341-04-01
    +5344-10-04 5342-08-04 5346-12-04 5346-12-04 5342-08-04 5342-08-04 5346-12-04
    +5397-07-13 5395-05-13 5399-09-13 5399-09-13 5395-05-13 5395-05-13 5399-09-13
    +5966-07-09 5964-05-09 5968-09-09 5968-09-09 5964-05-09 5964-05-09 5968-09-09
    +6229-06-28 6227-04-28 6231-08-28 6231-08-28 6227-04-28 6227-04-28 6231-08-28
    +6482-04-27 6480-02-27 6484-06-27 6484-06-27 6480-02-27 6480-02-27 6484-06-27
    +6631-11-13 6629-09-13 6634-01-13 6634-01-13 6629-09-13 6629-09-13 6634-01-13
    +6705-09-28 6703-07-28 6707-11-28 6707-11-28 6703-07-28 6703-07-28 6707-11-28
    +6731-02-12 6728-12-12 6733-04-12 6733-04-12 6728-12-12 6728-12-12 6733-04-12
    +7160-12-02 7158-10-02 7163-02-02 7163-02-02 7158-10-02 7158-10-02 7163-02-02
    +7409-09-07 7407-07-07 7411-11-07 7411-11-07 7407-07-07 7407-07-07 7411-11-07
    +7503-06-23 7501-04-23 7505-08-23 7505-08-23 7501-04-23 7501-04-23 7505-08-23
    +8422-07-22 8420-05-22 8424-09-22 8424-09-22 8420-05-22 8420-05-22 8424-09-22
    +8521-01-16 8518-11-16 8523-03-16 8523-03-16 8518-11-16 8518-11-16 8523-03-16
    +9075-06-13 9073-04-13 9077-08-13 9077-08-13 9073-04-13 9073-04-13 9077-08-13
    +9209-11-11 9207-09-11 9212-01-11 9212-01-11 9207-09-11 9207-09-11 9212-01-11
    +9403-01-09 9400-11-09 9405-03-09 9405-03-09 9400-11-09 9400-11-09 9405-03-09
    +PREHOOK: query: explain
    +select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), (dateval - 1999-06-07) (type: interval_day_time), (1999-06-07 - dateval) (type: interval_day_time), (dateval - dateval) (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time)
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: interval_day_time), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval c1 c2 c3
    +0004-09-22 -728552 23:00:00.000000000 728552 23:00:00.000000000 0 00:00:00.000000000
    +0528-10-27 -537126 23:00:00.000000000 537126 23:00:00.000000000 0 00:00:00.000000000
    +1319-02-02 -248481 23:00:00.000000000 248481 23:00:00.000000000 0 00:00:00.000000000
    +1404-07-23 -217263 23:00:00.000000000 217263 23:00:00.000000000 0 00:00:00.000000000
    +1815-05-06 -67236 23:00:00.000000000 67236 23:00:00.000000000 0 00:00:00.000000000
    +1883-04-17 -42418 23:00:00.000000000 42418 23:00:00.000000000 0 00:00:00.000000000
    +1966-08-16 -11983 00:00:00.000000000 11983 00:00:00.000000000 0 00:00:00.000000000
    +1973-04-17 -9546 23:00:00.000000000 9546 23:00:00.000000000 0 00:00:00.000000000
    +1974-10-04 -9012 00:00:00.000000000 9012 00:00:00.000000000 0 00:00:00.000000000
    +1976-03-03 -8495 23:00:00.000000000 8495 23:00:00.000000000 0 00:00:00.000000000
    +1976-05-06 -8432 00:00:00.000000000 8432 00:00:00.000000000 0 00:00:00.000000000
    +1978-08-05 -7611 00:00:00.000000000 7611 00:00:00.000000000 0 00:00:00.000000000
    +1981-04-25 -6616 23:00:00.000000000 6616 23:00:00.000000000 0 00:00:00.000000000
    +1981-11-15 -6412 23:00:00.000000000 6412 23:00:00.000000000 0 00:00:00.000000000
    +1985-07-20 -5070 00:00:00.000000000 5070 00:00:00.000000000 0 00:00:00.000000000
    +1985-11-18 -4948 23:00:00.000000000 4948 23:00:00.000000000 0 00:00:00.000000000
    +1987-02-21 -4488 23:00:00.000000000 4488 23:00:00.000000000 0 00:00:00.000000000
    +1987-05-28 -4393 00:00:00.000000000 4393 00:00:00.000000000 0 00:00:00.000000000
    +1998-10-16 -234 00:00:00.000000000 234 00:00:00.000000000 0 00:00:00.000000000
    +1999-10-03 118 00:00:00.000000000 -118 00:00:00.000000000 0 00:00:00.000000000
    +2000-12-18 560 01:00:00.000000000 -560 01:00:00.000000000 0 00:00:00.000000000
    +2002-05-10 1068 00:00:00.000000000 -1068 00:00:00.000000000 0 00:00:00.000000000
    +2003-09-23 1569 00:00:00.000000000 -1569 00:00:00.000000000 0 00:00:00.000000000
    +2004-03-07 1735 01:00:00.000000000 -1735 01:00:00.000000000 0 00:00:00.000000000
    +2007-02-09 2804 01:00:00.000000000 -2804 01:00:00.000000000 0 00:00:00.000000000
    +2009-01-21 3516 01:00:00.000000000 -3516 01:00:00.000000000 0 00:00:00.000000000
    +2010-04-08 3958 00:00:00.000000000 -3958 00:00:00.000000000 0 00:00:00.000000000
    +2013-04-07 5053 00:00:00.000000000 -5053 00:00:00.000000000 0 00:00:00.000000000
    +2013-04-10 5056 00:00:00.000000000 -5056 00:00:00.000000000 0 00:00:00.000000000
    +2021-09-24 8145 00:00:00.000000000 -8145 00:00:00.000000000 0 00:00:00.000000000
    +2024-11-11 9289 01:00:00.000000000 -9289 01:00:00.000000000 0 00:00:00.000000000
    +4143-07-08 783111 00:00:00.000000000 -783111 00:00:00.000000000 0 00:00:00.000000000
    +4966-12-04 1083855 01:00:00.000000000 -1083855 01:00:00.000000000 0 00:00:00.000000000
    +5339-02-01 1219784 01:00:00.000000000 -1219784 01:00:00.000000000 0 00:00:00.000000000
    +5344-10-04 1221856 00:00:00.000000000 -1221856 00:00:00.000000000 0 00:00:00.000000000
    +5397-07-13 1241131 00:00:00.000000000 -1241131 00:00:00.000000000 0 00:00:00.000000000
    +5966-07-09 1448949 00:00:00.000000000 -1448949 00:00:00.000000000 0 00:00:00.000000000
    +6229-06-28 1544997 00:00:00.000000000 -1544997 00:00:00.000000000 0 00:00:00.000000000
    +6482-04-27 1637342 00:00:00.000000000 -1637342 00:00:00.000000000 0 00:00:00.000000000
    +6631-11-13 1691962 01:00:00.000000000 -1691962 01:00:00.000000000 0 00:00:00.000000000
    +6705-09-28 1718944 00:00:00.000000000 -1718944 00:00:00.000000000 0 00:00:00.000000000
    +6731-02-12 1728212 01:00:00.000000000 -1728212 01:00:00.000000000 0 00:00:00.000000000
    +7160-12-02 1885195 01:00:00.000000000 -1885195 01:00:00.000000000 0 00:00:00.000000000
    +7409-09-07 1976054 00:00:00.000000000 -1976054 00:00:00.000000000 0 00:00:00.000000000
    +7503-06-23 2010310 00:00:00.000000000 -2010310 00:00:00.000000000 0 00:00:00.000000000
    +8422-07-22 2345998 00:00:00.000000000 -2345998 00:00:00.000000000 0 00:00:00.000000000
    +8521-01-16 2381970 01:00:00.000000000 -2381970 01:00:00.000000000 0 00:00:00.000000000
    +9075-06-13 2584462 00:00:00.000000000 -2584462 00:00:00.000000000 0 00:00:00.000000000
    +9209-11-11 2633556 01:00:00.000000000 -2633556 01:00:00.000000000 0 00:00:00.000000000
    +9403-01-09 2704106 01:00:00.000000000 -2704106 01:00:00.000000000 0 00:00:00.000000000
    +PREHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: tsval (type: timestamp), (tsval - 2-2) (type: timestamp), (tsval - -2-2) (type: timestamp), (tsval + 2-2) (type: timestamp), (tsval + -2-2) (type: timestamp), (-2-2 + tsval) (type: timestamp), (2-2 + tsval) (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: timestamp)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp)
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: timestamp), VALUE._col0 (type: timestamp), VALUE._col1 (type: timestamp), VALUE._col2 (type: timestamp), VALUE._col3 (type: timestamp), VALUE._col4 (type: timestamp), VALUE._col5 (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +tsval c1 c2 c3 c4 c5 c6
    +0004-09-22 18:26:29.519542222 0002-07-22 18:26:29.519542222 0006-11-22 18:26:29.519542222 0006-11-22 18:26:29.519542222 0002-07-22 18:26:29.519542222 0002-07-22 18:26:29.519542222 0006-11-22 18:26:29.519542222
    +0528-10-27 08:15:18.941718273 0526-08-27 08:15:18.941718273 0530-12-27 08:15:18.941718273 0530-12-27 08:15:18.941718273 0526-08-27 08:15:18.941718273 0526-08-27 08:15:18.941718273 0530-12-27 08:15:18.941718273
    +1319-02-02 16:31:57.778 1316-12-02 16:31:57.778 1321-04-02 16:31:57.778 1321-04-02 16:31:57.778 1316-12-02 16:31:57.778 1316-12-02 16:31:57.778 1321-04-02 16:31:57.778
    +1404-07-23 15:32:16.059185026 1402-05-23 15:32:16.059185026 1406-09-23 15:32:16.059185026 1406-09-23 15:32:16.059185026 1402-05-23 15:32:16.059185026 1402-05-23 15:32:16.059185026 1406-09-23 15:32:16.059185026
    +1815-05-06 00:12:37.543584705 1813-03-06 00:12:37.543584705 1817-07-06 00:12:37.543584705 1817-07-06 00:12:37.543584705 1813-03-06 00:12:37.543584705 1813-03-06 00:12:37.543584705 1817-07-06 00:12:37.543584705
    +1883-04-17 04:14:34.647766229 1881-02-17 04:14:34.647766229 1885-06-17 04:14:34.647766229 1885-06-17 04:14:34.647766229 1881-02-17 04:14:34.647766229 1881-02-17 04:14:34.647766229 1885-06-17 04:14:34.647766229
    +1966-08-16 13:36:50.183618031 1964-06-16 13:36:50.183618031 1968-10-16 13:36:50.183618031 1968-10-16 13:36:50.183618031 1964-06-16 13:36:50.183618031 1964-06-16 13:36:50.183618031 1968-10-16 13:36:50.183618031
    +1973-04-17 06:30:38.596784156 1971-02-17 06:30:38.596784156 1975-06-17 07:30:38.596784156 1975-06-17 07:30:38.596784156 1971-02-17 06:30:38.596784156 1971-02-17 06:30:38.596784156 1975-06-17 07:30:38.596784156
    +1974-10-04 17:21:03.989 1972-08-04 17:21:03.989 1976-12-04 16:21:03.989 1976-12-04 16:21:03.989 1972-08-04 17:21:03.989 1972-08-04 17:21:03.989 1976-12-04 16:21:03.989
    +1976-03-03 04:54:33.000895162 1974-01-03 04:54:33.000895162 1978-05-03 05:54:33.000895162 1978-05-03 05:54:33.000895162 1974-01-03 04:54:33.000895162 1974-01-03 04:54:33.000895162 1978-05-03 05:54:33.000895162
    +1976-05-06 00:42:30.910786948 1974-03-06 00:42:30.910786948 1978-07-06 00:42:30.910786948 1978-07-06 00:42:30.910786948 1974-03-06 00:42:30.910786948 1974-03-06 00:42:30.910786948 1978-07-06 00:42:30.910786948
    +1978-08-05 14:41:05.501 1976-06-05 14:41:05.501 1980-10-05 14:41:05.501 1980-10-05 14:41:05.501 1976-06-05 14:41:05.501 1976-06-05 14:41:05.501 1980-10-05 14:41:05.501
    +1981-04-25 09:01:12.077192689 1979-02-25 09:01:12.077192689 1983-06-25 10:01:12.077192689 1983-06-25 10:01:12.077192689 1979-02-25 09:01:12.077192689 1979-02-25 09:01:12.077192689 1983-06-25 10:01:12.077192689
    +1981-11-15 23:03:10.999338387 1979-09-16 00:03:10.999338387 1984-01-15 23:03:10.999338387 1984-01-15 23:03:10.999338387 1979-09-16 00:03:10.999338387 1979-09-16 00:03:10.999338387 1984-01-15 23:03:10.999338387
    +1985-07-20 09:30:11 1983-05-20 09:30:11 1987-09-20 09:30:11 1987-09-20 09:30:11 1983-05-20 09:30:11 1983-05-20 09:30:11 1987-09-20 09:30:11
    +1985-11-18 16:37:54 1983-09-18 17:37:54 1988-01-18 16:37:54 1988-01-18 16:37:54 1983-09-18 17:37:54 1983-09-18 17:37:54 1988-01-18 16:37:54
    +1987-02-21 19:48:29 1984-12-21 19:48:29 1989-04-21 20:48:29 1989-04-21 20:48:29 1984-12-21 19:48:29 1984-12-21 19:48:29 1989-04-21 20:48:29
    +1987-05-28 13:52:07.900916635 1985-03-28 12:52:07.900916635 1989-07-28 13:52:07.900916635 1989-07-28 13:52:07.900916635 1985-03-28 12:52:07.900916635 1985-03-28 12:52:07.900916635 1989-07-28 13:52:07.900916635
    +1998-10-16 20:05:29.397591987 1996-08-16 20:05:29.397591987 2000-12-16 19:05:29.397591987 2000-12-16 19:05:29.397591987 1996-08-16 20:05:29.397591987 1996-08-16 20:05:29.397591987 2000-12-16 19:05:29.397591987
    +1999-10-03 16:59:10.396903939 1997-08-03 16:59:10.396903939 2001-12-03 15:59:10.396903939 2001-12-03 15:59:10.396903939 1997-08-03 16:59:10.396903939 1997-08-03 16:59:10.396903939 2001-12-03 15:59:10.396903939
    +2000-12-18 08:42:30.000595596 1998-10-18 09:42:30.000595596 2003-02-18 08:42:30.000595596 2003-02-18 08:42:30.000595596 1998-10-18 09:42:30.000595596 1998-10-18 09:42:30.000595596 2003-02-18 08:42:30.000595596
    +2002-05-10 05:29:48.990818073 2000-03-10 04:29:48.990818073 2004-07-10 05:29:48.990818073 2004-07-10 05:29:48.990818073 2000-03-10 04:29:48.990818073 2000-03-10 04:29:48.990818073 2004-07-10 05:29:48.990818073
    +2003-09-23 22:33:17.00003252 2001-07-23 22:33:17.00003252 2005-11-23 21:33:17.00003252 2005-11-23 21:33:17.00003252 2001-07-23 22:33:17.00003252 2001-07-23 22:33:17.00003252 2005-11-23 21:33:17.00003252
    +2004-03-07 20:14:13 2002-01-07 20:14:13 2006-05-07 21:14:13 2006-05-07 21:14:13 2002-01-07 20:14:13 2002-01-07 20:14:13 2006-05-07 21:14:13
    +2007-02-09 05:17:29.368756876 2004-12-09 05:17:29.368756876 2009-04-09 06:17:29.368756876 2009-04-09 06:17:29.368756876 2004-12-09 05:17:29.368756876 2004-12-09 05:17:29.368756876 2009-04-09 06:17:29.368756876
    +2009-01-21 10:49:07.108 2006-11-21 10:49:07.108 2011-03-21 11:49:07.108 2011-03-21 11:49:07.108 2006-11-21 10:49:07.108 2006-11-21 10:49:07.108 2011-03-21 11:49:07.108
    +2010-04-08 02:43:35.861742727 2008-02-08 01:43:35.861742727 2012-06-08 02:43:35.861742727 2012-06-08 02:43:35.861742727 2008-02-08 01:43:35.861742727 2008-02-08 01:43:35.861742727 2012-06-08 02:43:35.861742727
    +2013-04-07 02:44:43.00086821 2011-02-07 01:44:43.00086821 2015-06-07 02:44:43.00086821 2015-06-07 02:44:43.00086821 2011-02-07 01:44:43.00086821 2011-02-07 01:44:43.00086821 2015-06-07 02:44:43.00086821
    +2013-04-10 00:43:46.854731546 2011-02-09 23:43:46.854731546 2015-06-10 00:43:46.854731546 2015-06-10 00:43:46.854731546 2011-02-09 23:43:46.854731546 2011-02-09 23:43:46.854731546 2015-06-10 00:43:46.854731546
    +2021-09-24 03:18:32.413655165 2019-07-24 03:18:32.413655165 2023-11-24 02:18:32.413655165 2023-11-24 02:18:32.413655165 2019-07-24 03:18:32.413655165 2019-07-24 03:18:32.413655165 2023-11-24 02:18:32.413655165
    +2024-11-11 16:42:41.101 2022-09-11 17:42:41.101 2027-01-11 16:42:41.101 2027-01-11 16:42:41.101 2022-09-11 17:42:41.101 2022-09-11 17:42:41.101 2027-01-11 16:42:41.101
    +4143-07-08 10:53:27.252802259 4141-05-08 10:53:27.252802259 4145-09-08 10:53:27.252802259 4145-09-08 10:53:27.252802259 4141-05-08 10:53:27.252802259 4141-05-08 10:53:27.252802259 4145-09-08 10:53:27.252802259
    +4966-12-04 09:30:55.202 4964-10-04 10:30:55.202 4969-02-04 09:30:55.202 4969-02-04 09:30:55.202 4964-10-04 10:30:55.202 4964-10-04 10:30:55.202 4969-02-04 09:30:55.202
    +5339-02-01 14:10:01.085678691 5336-12-01 14:10:01.085678691 5341-04-01 15:10:01.085678691 5341-04-01 15:10:01.085678691 5336-12-01 14:10:01.085678691 5336-12-01 14:10:01.085678691 5341-04-01 15:10:01.085678691
    +5344-10-04 18:40:08.165 5342-08-04 18:40:08.165 5346-12-04 17:40:08.165 5346-12-04 17:40:08.165 5342-08-04 18:40:08.165 5342-08-04 18:40:08.165 5346-12-04 17:40:08.165
    +5397-07-13 07:12:32.000896438 5395-05-13 07:12:32.000896438 5399-09-13 07:12:32.000896438 5399-09-13 07:12:32.000896438 5395-05-13 07:12:32.000896438 5395-05-13 07:12:32.000896438 5399-09-13 07:12:32.000896438
    +5966-07-09 03:30:50.597 5964-05-09 03:30:50.597 5968-09-09 03:30:50.597 5968-09-09 03:30:50.597 5964-05-09 03:30:50.597 5964-05-09 03:30:50.597 5968-09-09 03:30:50.597
    +6229-06-28 02:54:28.970117179 6227-04-28 02:54:28.970117179 6231-08-28 02:54:28.970117179 6231-08-28 02:54:28.970117179 6227-04-28 02:54:28.970117179 6227-04-28 02:54:28.970117179 6231-08-28 02:54:28.970117179
    +6482-04-27 12:07:38.073915413 6480-02-27 11:07:38.073915413 6484-06-27 12:07:38.073915413 6484-06-27 12:07:38.073915413 6480-02-27 11:07:38.073915413 6480-02-27 11:07:38.073915413 6484-06-27 12:07:38.073915413
    +6631-11-13 16:31:29.702202248 6629-09-13 17:31:29.702202248 6634-01-13 16:31:29.702202248 6634-01-13 16:31:29.702202248 6629-09-13 17:31:29.702202248 6629-09-13 17:31:29.702202248 6634-01-13 16:31:29.702202248
    +6705-09-28 18:27:28.000845672 6703-07-28 18:27:28.000845672 6707-11-28 17:27:28.000845672 6707-11-28 17:27:28.000845672 6703-07-28 18:27:28.000845672 6703-07-28 18:27:28.000845672 6707-11-28 17:27:28.000845672
    +6731-02-12 08:12:48.287783702 6728-12-12 08:12:48.287783702 6733-04-12 09:12:48.287783702 6733-04-12 09:12:48.287783702 6728-12-12 08:12:48.287783702 6728-12-12 08:12:48.287783702 6733-04-12 09:12:48.287783702
    +7160-12-02 06:00:24.81200852 7158-10-02 07:00:24.81200852 7163-02-02 06:00:24.81200852 7163-02-02 06:00:24.81200852 7158-10-02 07:00:24.81200852 7158-10-02 07:00:24.81200852 7163-02-02 06:00:24.81200852
    +7409-09-07 23:33:32.459349602 7407-07-07 23:33:32.459349602 7411-11-07 22:33:32.459349602 7411-11-07 22:33:32.459349602 7407-07-07 23:33:32.459349602 7407-07-07 23:33:32.459349602 7411-11-07 22:33:32.459349602
    +7503-06-23 23:14:17.486 7501-04-23 23:14:17.486 7505-08-23 23:14:17.486 7505-08-23 23:14:17.486 7501-04-23 23:14:17.486 7501-04-23 23:14:17.486 7505-08-23 23:14:17.486
    +8422-07-22 03:21:45.745036084 8420-05-22 03:21:45.745036084 8424-09-22 03:21:45.745036084 8424-09-22 03:21:45.745036084 8420-05-22 03:21:45.745036084 8420-05-22 03:21:45.745036084 8424-09-22 03:21:45.745036084
    +8521-01-16 20:42:05.668832388 8518-11-16 20:42:05.668832388 8523-03-16 21:42:05.668832388 8523-03-16 21:42:05.668832388 8518-11-16 20:42:05.668832388 8518-11-16 20:42:05.668832388 8523-03-16 21:42:05.668832388
    +9075-06-13 16:20:09.218517797 9073-04-13 16:20:09.218517797 9077-08-13 16:20:09.218517797 9077-08-13 16:20:09.218517797 9073-04-13 16:20:09.218517797 9073-04-13 16:20:09.218517797 9077-08-13 16:20:09.218517797
    +9209-11-11 04:08:58.223768453 9207-09-11 05:08:58.223768453 9212-01-11 04:08:58.223768453 9212-01-11 04:08:58.223768453 9207-09-11 05:08:58.223768453 9207-09-11 05:08:58.223768453 9212-01-11 04:08:58.223768453
    +9403-01-09 18:12:33.547 9400-11-09 18:12:33.547 9405-03-09 18:12:33.547 9405-03-09 18:12:33.547 9400-11-09 18:12:33.547 9400-11-09 18:12:33.547 9405-03-09 18:12:33.547
    +PREHOOK: query: explain
    +select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
    + Select Operator
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + Reduce Output Operator
    + key expressions: 5-5 (type: interval_year_month)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + TopN Hash Memory Usage: 0.1
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month)
    + outputColumnNames: _col0, _col1
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + Limit
    + Number of rows: 2
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: 2
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +c0 c1
    +5-5 -1-1
    +5-5 -1-1
    +PREHOOK: query: -- interval day-time arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: -- interval day-time arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), (dateval - 99 11:22:33.123456789) (type: timestamp), (dateval - -99 11:22:33.123456789) (type: timestamp), (dateval + 99 11:22:33.123456789) (type: timestamp), (dateval + -99 11:22:33.123456789) (type: timestamp), (-99 11:22:33.123456789 + dateval) (type: timestamp), (99 11:22:33.123456789 + dateval) (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp)
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: timestamp), VALUE._col1 (type: timestamp), VALUE._col2 (type: timestamp), VALUE._col3 (type: timestamp), VALUE._col4 (type: timestamp), VALUE._col5 (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval _c1 _c2 _c3 _c4 _c5 _c6
    +0004-09-22 0004-06-14 12:37:26.876543211 0004-12-30 11:22:33.123456789 0004-12-30 11:22:33.123456789 0004-06-14 12:37:26.876543211 0004-06-14 12:37:26.876543211 0004-12-30 11:22:33.123456789
    +0528-10-27 0528-07-19 12:37:26.876543211 0529-02-03 11:22:33.123456789 0529-02-03 11:22:33.123456789 0528-07-19 12:37:26.876543211 0528-07-19 12:37:26.876543211 0529-02-03 11:22:33.123456789
    +1319-02-02 1318-10-25 12:37:26.876543211 1319-05-12 11:22:33.123456789 1319-05-12 11:22:33.123456789 1318-10-25 12:37:26.876543211 1318-10-25 12:37:26.876543211 1319-05-12 11:22:33.123456789
    +1404-07-23 1404-04-14 12:37:26.876543211 1404-10-30 11:22:33.123456789 1404-10-30 11:22:33.123456789 1404-04-14 12:37:26.876543211 1404-04-14 12:37:26.876543211 1404-10-30 11:22:33.123456789
    +1815-05-06 1815-01-26 12:37:26.876543211 1815-08-13 11:22:33.123456789 1815-08-13 11:22:33.123456789 1815-01-26 12:37:26.876543211 1815-01-26 12:37:26.876543211 1815-08-13 11:22:33.123456789
    +1883-04-17 1883-01-07 12:37:26.876543211 1883-07-25 11:22:33.123456789 1883-07-25 11:22:33.123456789 1883-01-07 12:37:26.876543211 1883-01-07 12:37:26.876543211 1883-07-25 11:22:33.123456789
    +1966-08-16 1966-05-08 12:37:26.876543211 1966-11-23 10:22:33.123456789 1966-11-23 10:22:33.123456789 1966-05-08 12:37:26.876543211 1966-05-08 12:37:26.876543211 1966-11-23 10:22:33.123456789
    +1973-04-17 1973-01-07 12:37:26.876543211 1973-07-25 12:22:33.123456789 1973-07-25 12:22:33.123456789 1973-01-07 12:37:26.876543211 1973-01-07 12:37:26.876543211 1973-07-25 12:22:33.123456789
    +1974-10-04 1974-06-26 12:37:26.876543211 1975-01-11 10:22:33.123456789 1975-01-11 10:22:33.123456789 1974-06-26 12:37:26.876543211 1974-06-26 12:37:26.876543211 1975-01-11 10:22:33.123456789
    +1976-03-03 1975-11-24 12:37:26.876543211 1976-06-10 12:22:33.123456789 1976-06-10 12:22:33.123456789 1975-11-24 12:37:26.876543211 1975-11-24 12:37:26.876543211 1976-06-10 12:22:33.123456789
    +1976-05-06 1976-01-27 11:37:26.876543211 1976-08-13 11:22:33.123456789 1976-08-13 11:22:33.123456789 1976-01-27 11:37:26.876543211 1976-01-27 11:37:26.876543211 1976-08-13 11:22:33.123456789
    +1978-08-05 1978-04-27 11:37:26.876543211 1978-11-12 10:22:33.123456789 1978-11-12 10:22:33.123456789 1978-04-27 11:37:26.876543211 1978-04-27 11:37:26.876543211 1978-11-12 10:22:33.123456789
    +1981-04-25 1981-01-15 12:37:26.876543211 1981-08-02 12:22:33.123456789 1981-08-02 12:22:33.123456789 1981-01-15 12:37:26.876543211 1981-01-15 12:37:26.876543211 1981-08-02 12:22:33.123456789
    +1981-11-15 1981-08-07 13:37:26.876543211 1982-02-22 11:22:33.123456789 1982-02-22 11:22:33.123456789 1981-08-07 13:37:26.876543211 1981-08-07 13:37:26.876543211 1982-02-22 11:22:33.123456789
    +1985-07-20 1985-04-11 11:37:26.876543211 1985-10-27 10:22:33.123456789 1985-10-27 10:22:33.123456789 1985-04-11 11:37:26.876543211 1985-04-11 11:37:26.876543211 1985-10-27 10:22:33.123456789
    +1985-11-18 1985-08-10 13:37:26.876543211 1986-02-25 11:22:33.123456789 1986-02-25 11:22:33.123456789 1985-08-10 13:37:26.876543211 1985-08-10 13:37:26.876543211 1986-02-25 11:22:33.123456789
    +1987-02-21 1986-11-13 12:37:26.876543211 1987-05-31 12:22:33.123456789 1987-05-31 12:22:33.123456789 1986-11-13 12:37:26.876543211 1986-11-13 12:37:26.876543211 1987-05-31 12:22:33.123456789
    +1987-05-28 1987-02-17 11:37:26.876543211 1987-09-04 11:22:33.123456789 1987-09-04 11:22:33.123456789 1987-02-17 11:37:26.876543211 1987-02-17 11:37:26.876543211 1987-09-04 11:22:33.123456789
    +1998-10-16 1998-07-08 12:37:26.876543211 1999-01-23 10:22:33.123456789 1999-01-23 10:22:33.123456789 1998-07-08 12:37:26.876543211 1998-07-08 12:37:26.876543211 1999-01-23 10:22:33.123456789
    +1999-10-03 1999-06-25 12:37:26.876543211 2000-01-10 10:22:33.123456789 2000-01-10 10:22:33.123456789 1999-06-25 12:37:26.876543211 1999-06-25 12:37:26.876543211 2000-01-10 10:22:33.123456789
    +2000-12-18 2000-09-09 13:37:26.876543211 2001-03-27 11:22:33.123456789 2001-03-27 11:22:33.123456789 2000-09-09 13:37:26.876543211 2000-09-09 13:37:26.876543211 2001-03-27 11:22:33.123456789
    +2002-05-10 2002-01-30 11:37:26.876543211 2002-08-17 11:22:33.123456789 2002-08-17 11:22:33.123456789 2002-01-30 11:37:26.876543211 2002-01-30 11:37:26.876543211 2002-08-17 11:22:33.123456789
    +2003-09-23 2003-06-15 12:37:26.876543211 2003-12-31 10:22:33.123456789 2003-12-31 10:22:33.123456789 2003-06-15 12:37:26.876543211 2003-06-15 12:37:26.876543211 2003-12-31 10:22:33.123456789
    +2004-03-07 2003-11-28 12:37:26.876543211 2004-06-14 12:22:33.123456789 2004-06-14 12:22:33.123456789 2003-11-28 12:37:26.876543211 2003-11-28 12:37:26.876543211 2004-06-14 12:22:33.123456789
    +2007-02-09 2006-11-01 12:37:26.876543211 2007-05-19 12:22:33.123456789 2007-05-19 12:22:33.123456789 2006-11-01 12:37:26.876543211 2006-11-01 12:37:26.876543211 2007-05-19 12:22:33.123456789
    +2009-01-21 2008-10-13 13:37:26.876543211 2009-04-30 12:22:33.123456789 2009-04-30 12:22:33.123456789 2008-10-13 13:37:26.876543211 2008-10-13 13:37:26.876543211 2009-04-30 12:22:33.123456789
    +2010-04-08 2009-12-29 11:37:26.876543211 2010-07-16 11:22:33.123456789 2010-07-16 11:22:33.123456789 2009-12-29 11:37:26.876543211 2009-12-29 11:37:26.876543211 2010-07-16 11:22:33.123456789
    +2013-04-07 2012-12-28 11:37:26.876543211 2013-07-15 11:22:33.123456789 2013-07-15 11:22:33.123456789 2012-12-28 11:37:26.876543211 2012-12-28 11:37:26.876543211 2013-07-15 11:22:33.123456789
    +2013-04-10 2012-12-31 11:37:26.876543211 2013-07-18 11:22:33.123456789 2013-07-18 11:22:33.123456789 2012-12-31 11:37:26.876543211 2012-12-31 11:37:26.876543211 2013-07-18 11:22:33.123456789
    +2021-09-24 2021-06-16 12:37:26.876543211 2022-01-01 10:22:33.123456789 2022-01-01 10:22:33.123456789 2021-06-16 12:37:26.876543211 2021-06-16 12:37:26.876543211 2022-01-01 10:22:33.123456789
    +2024-11-11 2024-08-03 13:37:26.876543211 2025-02-18 11:22:33.123456789 2025-02-18 11:22:33.123456789 2024-08-03 13:37:26.876543211 2024-08-03 13:37:26.876543211 2025-02-18 11:22:33.123456789
    +4143-07-08 4143-03-30 12:37:26.876543211 4143-10-15 11:22:33.123456789 4143-10-15 11:22:33.123456789 4143-03-30 12:37:26.876543211 4143-03-30 12:37:26.876543211 4143-10-15 11:22:33.123456789
    +4966-12-04 4966-08-26 13:37:26.876543211 4967-03-13 12:22:33.123456789 4967-03-13 12:22:33.123456789 4966-08-26 13:37:26.876543211 4966-08-26 13:37:26.876543211 4967-03-13 12:22:33.123456789
    +5339-02-01 5338-10-24 13:37:26.876543211 5339-05-11 12:22:33.123456789 5339-05-11 12:22:33.123456789 5338-10-24 13:37:26.876543211 5338-10-24 13:37:26.876543211 5339-05-11 12:22:33.123456789
    +5344-10-04 5344-06-26 12:37:26.876543211 5345-01-11 10:22:33.123456789 5345-01-11 10:22:33.123456789 5344-06-26 12:37:26.876543211 5344-06-26 12:37:26.876543211 5345-01-11 10:22:33.123456789
    +5397-07-13 5397-04-04 12:37:26.876543211 5397-10-20 11:22:33.123456789 5397-10-20 11:22:33.123456789 5397-04-04 12:37:26.876543211 5397-04-04 12:37:26.876543211 5397-10-20 11:22:33.123456789
    +5966-07-09 5966-03-31 12:37:26.876543211 5966-10-16 11:22:33.123456789 5966-10-16 11:22:33.123456789 5966-03-31 12:37:26.876543211 5966-03-31 12:37:26.876543211 5966-10-16 11:22:33.123456789
    +6229-06-28 6229-03-20 12:37:26.876543211 6229-10-05 11:22:33.123456789 6229-10-05 11:22:33.123456789 6229-03-20 12:37:26.876543211 6229-03-20 12:37:26.876543211 6229-10-05 11:22:33.123456789
    +6482-04-27 6482-01-17 11:37:26.876543211 6482-08-04 11:22:33.123456789 6482-08-04 11:22:33.123456789 6482-01-17 11:37:26.876543211 6482-01-17 11:37:26.876543211 6482-08-04 11:22:33.123456789
    +6631-11-13 6631-08-05 13:37:26.876543211 6632-02-20 11:22:33.123456789 6632-02-20 11:22:33.123456789 6631-08-05 13:37:26.876543211 6631-08-05 13:37:26.876543211 6632-02-20 11:22:33.123456789
    +6705-09-28 6705-06-20 12:37:26.876543211 6706-01-05 10:22:33.123456789 6706-01-05 10:22:33.123456789 6705-06-20 12:37:26.876543211 6705-06-20 12:37:26.876543211 6706-01-05 10:22:33.123456789
    +6731-02-12 6730-11-04 12:37:26.876543211 6731-05-22 12:22:33.123456789 6731-05-22 12:22:33.123456789 6730-11-04 12:37:26.876543211 6730-11-04 12:37:26.876543211 6731-05-22 12:22:33.123456789
    +7160-12-02 7160-08-24 13:37:26.876543211 7161-03-11 11:22:33.123456789 7161-03-11 11:22:33.123456789 7160-08-24 13:37:26.876543211 7160-08-24 13:37:26.876543211 7161-03-11 11:22:33.123456789
    +7409-09-07 7409-05-30 12:37:26.876543211 7409-12-15 10:22:33.123456789 7409-12-15 10:22:33.123456789 7409-05-30 12:37:26.876543211 7409-05-30 12:37:26.876543211 7409-12-15 10:22:33.123456789
    +7503-06-23 7503-03-15 12:37:26.876543211 7503-09-30 11:22:33.123456789 7503-09-30 11:22:33.123456789 7503-03-15 12:37:26.876543211 7503-03-15 12:37:26.876543211 7503-09-30 11:22:33.123456789
    +8422-07-22 8422-04-13 12:37:26.876543211 8422-10-29 11:22:33.123456789 8422-10-29 11:22:33.123456789 8422-04-13 12:37:26.876543211 8422-04-13 12:37:26.876543211 8422-10-29 11:22:33.123456789
    +8521-01-16 8520-10-08 13:37:26.876543211 8521-04-25 12:22:33.123456789 8521-04-25 12:22:33.123456789 8520-10-08 13:37:26.876543211 8520-10-08 13:37:26.876543211 8521-04-25 12:22:33.123456789
    +9075-06-13 9075-03-05 11:37:26.876543211 9075-09-20 11:22:33.123456789 9075-09-20 11:22:33.123456789 9075-03-05 11:37:26.876543211 9075-03-05 11:37:26.876543211 9075-09-20 11:22:33.123456789
    +9209-11-11 9209-08-03 13:37:26.876543211 9210-02-18 11:22:33.123456789 9210-02-18 11:22:33.123456789 9209-08-03 13:37:26.876543211 9209-08-03 13:37:26.876543211 9210-02-18 11:22:33.123456789
    +9403-01-09 9402-10-01 13:37:26.876543211 9403-04-18 12:22:33.123456789 9403-04-18 12:22:33.123456789 9402-10-01 13:37:26.876543211 9402-10-01 13:37:26.876543211 9403-04-18 12:22:33.123456789
    +PREHOOK: query: explain
    +select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), tsval (type: timestamp), (dateval - tsval) (type: interval_day_time), (tsval - dateval) (type: interval_day_time), (tsval - tsval) (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col4 (type: interval_day_time)
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: timestamp), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval tsval c2 c3 c4
    +0004-09-22 0004-09-22 18:26:29.519542222 -0 18:26:30.519542222 0 18:26:30.519542222 0 00:00:00.000000000
    +0528-10-27 0528-10-27 08:15:18.941718273 -0 08:15:19.941718273 0 08:15:19.941718273 0 00:00:00.000000000
    +1319-02-02 1319-02-02 16:31:57.778 -0 16:31:58.778000000 0 16:31:58.778000000 0 00:00:00.000000000
    +1404-07-23 1404-07-23 15:32:16.059185026 -0 15:32:17.059185026 0 15:32:17.059185026 0 00:00:00.000000000
    +1815-05-06 1815-05-06 00:12:37.543584705 -0 00:12:38.543584705 0 00:12:38.543584705 0 00:00:00.000000000
    +1883-04-17 1883-04-17 04:14:34.647766229 -0 04:14:35.647766229 0 04:14:35.647766229 0 00:00:00.000000000
    +1966-08-16 1966-08-16 13:36:50.183618031 -0 13:36:51.183618031 0 13:36:51.183618031 0 00:00:00.000000000
    +1973-04-17 1973-04-17 06:30:38.596784156 -0 06:30:38.596784156 0 06:30:38.596784156 0 00:00:00.000000000
    +1974-10-04 1974-10-04 17:21:03.989 -0 17:21:03.989000000 0 17:21:03.989000000 0 00:00:00.000000000
    +1976-03-03 1976-03-03 04:54:33.000895162 -0 04:54:33.000895162 0 04:54:33.000895162 0 00:00:00.000000000
    +1976-05-06 1976-05-06 00:42:30.910786948 -0 00:42:30.910786948 0 00:42:30.910786948 0 00:00:00.000000000
    +1978-08-05 1978-08-05 14:41:05.501 -0 14:41:05.501000000 0 14:41:05.501000000 0 00:00:00.000000000
    +1981-04-25 1981-04-25 09:01:12.077192689 -0 09:01:12.077192689 0 09:01:12.077192689 0 00:00:00.000000000
    +1981-11-15 1981-11-15 23:03:10.999338387 -0 23:03:10.999338387 0 23:03:10.999338387 0 00:00:00.000000000
    +1985-07-20 1985-07-20 09:30:11 -0 09:30:11.000000000 0 09:30:11.000000000 0 00:00:00.000000000
    +1985-11-18 1985-11-18 16:37:54 -0 16:37:54.000000000 0 16:37:54.000000000 0 00:00:00.000000000
    +1987-02-21 1987-02-21 19:48:29 -0 19:48:29.000000000 0 19:48:29.000000000 0 00:00:00.000000000
    +1987-05-28 1987-05-28 13:52:07.900916635 -0 13:52:07.900916635 0 13:52:07.900916635 0 00:00:00.000000000
    +1998-10-16 1998-10-16 20:05:29.397591987 -0 20:05:29.397591987 0 20:05:29.397591987 0 00:00:00.000000000
    +1999-10-03 1999-10-03 16:59:10.396903939 -0 16:59:10.396903939 0 16:59:10.396903939 0 00:00:00.000000000
    +2000-12-18 2000-12-18 08:42:30.000595596 -0 08:42:30.000595596 0 08:42:30.000595596 0 00:00:00.000000000
    +2002-05-10 2002-05-10 05:29:48.990818073 -0 05:29:48.990818073 0 05:29:48.990818073 0 00:00:00.000000000
    +2003-09-23 2003-09-23 22:33:17.00003252 -0 22:33:17.000032520 0 22:33:17.000032520 0 00:00:00.000000000
    +2004-03-07 2004-03-07 20:14:13 -0 20:14:13.000000000 0 20:14:13.000000000 0 00:00:00.000000000
    +2007-02-09 2007-02-09 05:17:29.368756876 -0 05:17:29.368756876 0 05:17:29.368756876 0 00:00:00.000000000
    +2009-01-21 2009-01-21 10:49:07.108 -0 10:49:07.108000000 0 10:49:07.108000000 0 00:00:00.000000000
    +2010-04-08 2010-04-08 02:43:35.861742727 -0 02:43:35.861742727 0 02:43:35.861742727 0 00:00:00.000000000
    +2013-04-07 2013-04-07 02:44:43.00086821 -0 02:44:43.000868210 0 02:44:43.000868210 0 00:00:00.000000000
    +2013-04-10 2013-04-10 00:43:46.854731546 -0 00:43:46.854731546 0 00:43:46.854731546 0 00:00:00.000000000
    +2021-09-24 2021-09-24 03:18:32.413655165 -0 03:18:32.413655165 0 03:18:32.413655165 0 00:00:00.000000000
    +2024-11-11 2024-11-11 16:42:41.101 -0 16:42:41.101000000 0 16:42:41.101000000 0 00:00:00.000000000
    +4143-07-08 4143-07-08 10:53:27.252802259 -0 10:53:27.252802259 0 10:53:27.252802259 0 00:00:00.000000000
    +4966-12-04 4966-12-04 09:30:55.202 -0 09:30:55.202000000 0 09:30:55.202000000 0 00:00:00.000000000
    +5339-02-01 5339-02-01 14:10:01.085678691 -0 14:10:01.085678691 0 14:10:01.085678691 0 00:00:00.000000000
    +5344-10-04 5344-10-04 18:40:08.165 -0 18:40:08.165000000 0 18:40:08.165000000 0 00:00:00.000000000
    +5397-07-13 5397-07-13 07:12:32.000896438 -0 07:12:32.000896438 0 07:12:32.000896438 0 00:00:00.000000000
    +5966-07-09 5966-07-09 03:30:50.597 -0 03:30:50.597000000 0 03:30:50.597000000 0 00:00:00.000000000
    +6229-06-28 6229-06-28 02:54:28.970117179 -0 02:54:28.970117179 0 02:54:28.970117179 0 00:00:00.000000000
    +6482-04-27 6482-04-27 12:07:38.073915413 -0 12:07:38.073915413 0 12:07:38.073915413 0 00:00:00.000000000
    +6631-11-13 6631-11-13 16:31:29.702202248 -0 16:31:29.702202248 0 16:31:29.702202248 0 00:00:00.000000000
    +6705-09-28 6705-09-28 18:27:28.000845672 -0 18:27:28.000845672 0 18:27:28.000845672 0 00:00:00.000000000
    +6731-02-12 6731-02-12 08:12:48.287783702 -0 08:12:48.287783702 0 08:12:48.287783702 0 00:00:00.000000000
    +7160-12-02 7160-12-02 06:00:24.81200852 -0 06:00:24.812008520 0 06:00:24.812008520 0 00:00:00.000000000
    +7409-09-07 7409-09-07 23:33:32.459349602 -0 23:33:32.459349602 0 23:33:32.459349602 0 00:00:00.000000000
    +7503-06-23 7503-06-23 23:14:17.486 -0 23:14:17.486000000 0 23:14:17.486000000 0 00:00:00.000000000
    +8422-07-22 8422-07-22 03:21:45.745036084 -0 03:21:45.745036084 0 03:21:45.745036084 0 00:00:00.000000000
    +8521-01-16 8521-01-16 20:42:05.668832388 -0 20:42:05.668832388 0 20:42:05.668832388 0 00:00:00.000000000
    +9075-06-13 9075-06-13 16:20:09.218517797 -0 16:20:09.218517797 0 16:20:09.218517797 0 00:00:00.000000000
    +9209-11-11 9209-11-11 04:08:58.223768453 -0 04:08:58.223768453 0 04:08:58.223768453 0 00:00:00.000000000
    +9403-01-09 9403-01-09 18:12:33.547 -0 18:12:33.547000000 0 18:12:33.547000000 0 00:00:00.000000000
    +PREHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: tsval (type: timestamp), (tsval - 99 11:22:33.123456789) (type: timestamp), (tsval - -99 11:22:33.123456789) (type: timestamp), (tsval + 99 11:22:33.123456789) (type: timestamp), (tsval + -99 11:22:33.123456789) (type: timestamp), (-99 11:22:33.123456789 + tsval) (type: timestamp), (99 11:22:33.123456789 + tsval) (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: timestamp)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp)
    + Execution mode: vectorized
    + Reducer 2
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: timestamp), VALUE._col0 (type: timestamp), VALUE._col1 (type: timestamp), VALUE._col2 (type: timestamp), VALUE._col3 (type: timestamp), VALUE._col4 (type: timestamp), VALUE._col5 (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +tsval _c1 _c2 _c3 _c4 _c5 _c6
    +0004-09-22 18:26:29.519542222 0004-06-15 07:03:56.396085433 0004-12-31 05:49:02.642999011 0004-12-31 05:49:02.642999011 0004-06-15 07:03:56.396085433 0004-06-15 07:03:56.396085433 0004-12-31 05:49:02.642999011
    +0528-10-27 08:15:18.941718273 0528-07-19 20:52:45.818261484 0529-02-03 19:37:52.065175062 0529-02-03 19:37:52.065175062 0528-07-19 20:52:45.818261484 0528-07-19 20:52:45.818261484 0529-02-03 19:37:52.065175062
    +1319-02-02 16:31:57.778 1318-10-26 05:09:24.654543211 1319-05-13 03:54:30.901456789 1319-05-13 03:54:30.901456789 1318-10-26 05:09:24.654543211 1318-10-26 05:09:24.654543211 1319-05-13 03:54:30.901456789
    +1404-07-23 15:32:16.059185026 1404-04-15 04:09:42.935728237 1404-10-31 02:54:49.182641815 1404-10-31 02:54:49.182641815 1404-04-15 04:09:42.935728237 1404-04-15 04:09:42.935728237 1404-10-31 02:54:49.182641815
    +1815-05-06 00:12:37.543584705 1815-01-26 12:50:04.420127916 1815-08-13 11:35:10.667041494 1815-08-13 11:35:10.667041494 1815-01-26 12:50:04.420127916 1815-01-26 12:50:04.420127916 1815-08-13 11:35:10.667041494
    +1883-04-17 04:14:34.647766229 1883-01-07 16:52:01.52430944 1883-07-25 15:37:07.771223018 1883-07-25 15:37:07.771223018 1883-01-07 16:52:01.52430944 1883-01-07 16:52:01.52430944 1883-07-25 15:37:07.771223018
    +1966-08-16 13:36:50.183618031 1966-05-09 02:14:17.060161242 1966-11-23 23:59:23.30707482 1966-11-23 23:59:23.30707482 1966-05-09 02:14:17.060161242 1966-05-09 02:14:17.060161242 1966-11-23 23:59:23.30707482
    +1973-04-17 06:30:38.596784156 1973-01-07 19:08:05.473327367 1973-07-25 18:53:11.720240945 1973-07-25 18:53:11.720240945 1973-01-07 19:08:05.473327367 1973-01-07 19:08:05.473327367 1973-07-25 18:53:11.720240945
    +1974-10-04 17:21:03.989 1974-06-27 05:58:30.865543211 1975-01-12 03:43:37.112456789 1975-01-12 03:43:37.112456789 1974-06-27 05:58:30.865543211 1974-06-27 05:58:30.865543211 1975-01-12 03:43:37.112456789
    +1976-03-03 04:54:33.000895162 1975-11-24 17:31:59.877438373 1976-06-10 17:17:06.124351951 1976-06-10 17:17:06.124351951 1975-11-24 17:31:59.877438373 1975-11-24 17:31:59.877438373 1976-06-10 17:17:06.124351951
    +1976-05-06 00:42:30.910786948 1976-01-27 12:19:57.787330159 1976-08-13 12:05:04.034243737 1976-08-13 12:05:04.034243737 1976-01-27 12:19:57.787330159 1976-01-27 12:19:57.787330159 1976-08-13 12:05:04.034243737
    +1978-08-05 14:41:05.501 1978-04-28 02:18:32.377543211 1978-11-13 01:03:38.624456789 1978-11-13 01:03:38.624456789 1978-04-28 02:18:32.377543211 1978-04-28 02:18:32.377543211 1978-11-13 01:03:38.624456789
    +1981-04-25 09:01:12.077192689 1981-01-15 21:38:38.9537359 1981-08-02 21:23:45.200649478 1981-08-02 21:23:45.200649478 1981-01-15 21:38:38.9537359 1981-01-15 21:38:38.9537359 1981-08-02 21:23:45.200649478
    +1981-11-15 23:03:10.999338387 1981-08-08 12:40:37.875881598 1982-02-23 10:25:44.122795176 1982-02-23 10:25:44.122795176 1981-08-08 12:40:37.875881598 1981-08-08 12:40:37.875881598 1982-02-23 10:25:44.122795176
    +1985-07-20 09:30:11 1985-04-11 21:07:37.876543211 1985-10-27 19:52:44.123456789 1985-10-27 19:52:44.123456789 1985-04-11 21:07:37.876543211 1985-04-11 21:07:37.876543211 1985-10-27 19:52:44.123456789
    +1985-11-18 16:37:54 1985-08-11 06:15:20.876543211 1986-02-26 04:00:27.123456789 1986-02-26 04:00:27.123456789 1985-08-11 06:15:20.876543211 1985-08-11 06:15:20.876543211 1986-02-26 04:00:27.123456789
    +1987-02-21 19:48:29 1986-11-14 08:25:55.876543211 1987-06-01 08:11:02.123456789 1987-06-01 08:11:02.123456789 1986-11-14 08:25:55.876543211 1986-11-14 08:25:55.876543211 1987-06-01 08:11:02.123456789
    +1987-05-28 13:52:07.900916635 1987-02-18 01:29:34.777459846 1987-09-05 01:14:41.024373424 1987-09-05 01:14:41.024373424 1987-02-18 01:29:34.777459846 1987-02-18 01:29:34.777459846 1987-09-05 01:14:41.024373424
    +1998-10-16 20:05:29.397591987 1998-07-09 08:42:56.274135198 1999-01-24 06:28:02.521048776 1999-01-24 06:28:02.521048776 1998-07-09 08:42:56.274135198 1998-07-09 08:42:56.274135198 1999-01-24 06:28:02.521048776
    +1999-10-03 16:59:10.396903939 1999-06-26 05:36:37.27344715 2000-01-11 03:21:43.520360728 2000-01-11 03:21:43.520360728 1999-06-26 05:36:37.27344715 1999-06-26 05:36:37.27344715 2000-01-11 03:21:43.520360728
    +2000-12-18 08:42:30.000595596 2000-09-09 22:19:56.877138807 2001-03-27 20:05:03.124052385 2001-03-27 20:05:03.124052385 2000-09-09 22:19:56.877138807 2000-09-09 22:19:56.877138807 2001-03-27 20:05:03.124052385
    +2002-05-10 05:29:48.990818073 2002-01-30 17:07:15.867361284 2002-08-17 16:52:22.114274862 2002-08-17 16:52:22.114274862 2002-01-30 17:07:15.867361284 2002-01-30 17:07:15.867361284 2002-08-17 16:52:22.114274862
    +2003-09-23 22:33:17.00003252 2003-06-16 11:10:43.876575731 2004-01-01 08:55:50.123489309 2004-01-01 08:55:50.123489309 2003-06-16 11:10:43.876575731 2003-06-16 11:10:43.876575731 2004-01-01 08:55:50.123489309
    +2004-03-07 20:14:13 2003-11-29 08:51:39.876543211 2004-06-15 08:36:46.123456789 2004-06-15 08:36:46.123456789 2003-11-29 08:51:39.876543211 2003-11-29 08:51:39.876543211 2004-06-15 08:36:46.123456789
    +2007-02-09 05:17:29.368756876 2006-11-01 17:54:56.245300087 2007-05-19 17:40:02.492213665 2007-05-19 17:40:02.492213665 2006-11-01 17:54:56.245300087 2006-11-01 17:54:56.245300087 2007-05-19 17:40:02.492213665
    +2009-01-21 10:49:07.108 2008-10-14 00:26:33.984543211 2009-04-30 23:11:40.231456789 2009-04-30 23:11:40.231456789 2008-10-14 00:26:33.984543211 2008-10-14 00:26:33.984543211 2009-04-30 23:11:40.231456789
    +2010-04-08 02:43:35.861742727 2009-12-29 14:21:02.738285938 2010-07-16 14:06:08.985199516 2010-07-16 14:06:08.985199516 2009-12-29 14:21:02.738285938 2009-12-29 14:21:02.738285938 2010-07-16 14:06:08.985199516
    +2013-04-07 02:44:43.00086821 2012-12-28 14:22:09.877411421 2013-07-15 14:07:16.124324999 2013-07-15 14:07:16.124324999 2012-12-28 14:22:09.877411421 2012-12-28 14:22:09.877411421 2013-07-15 14:07:16.124324999
    +2013-04-10 00:43:46.854731546 2012-12-31 12:21:13.731274757 2013-07-18 12:06:19.978188335 2013-07-18 12:06:19.978188335 2012-12-31 12:21:13.731274757 2012-12-31 12:21:13.731274757 2013-07-18 12:06:19.978188335
    +2021-09-24 03:18:32.413655165 2021-06-16 15:55:59.290198376 2022-01-01 13:41:05.537111954 2022-01-01 13:41:05.537111954 2021-06-16 15:55:59.290198376 2021-06-16 15:55:59.290198376 2022-01-01 13:41:05.537111954
    +2024-11-11 16:42:41.101 2024-08-04 06:20:07.977543211 2025-02-19 04:05:14.224456789 2025-02-19 04:05:14.224456789 2024-08-04 06:20:07.977543211 2024-08-04 06:20:07.977543211 2025-02-19 04:05:14.224456789
    +4143-07-08 10:53:27.252802259 4143-03-30 23:30:54.12934547 4143-10-15 22:16:00.376259048 4143-10-15 22:16:00.376259048 4143-03-30 23:30:54.12934547 4143-03-30 23:30:54.12934547 4143-10-15 22:16:00.376259048
    +4966-12-04 09:30:55.202 4966-08-26 23:08:22.078543211 4967-03-13 21:53:28.325456789 4967-03-13 21:53:28.325456789 4966-08-26 23:08:22.078543211 4966-08-26 23:08:22.078543211 4967-03-13 21:53:28.325456789
    +5339-02-01 14:10:01.085678691 5338-10-25 03:47:27.962221902 5339-05-12 02:32:34.20913548 5339-05-12 02:32:34.20913548 5338-10-25 03:47:27.962221902 5338-10-25 03:47:27.962221902 5339-05-12 02:32:34.20913548
    +5344-10-04 18:40:08.165 5344-06-27 07:17:35.041543211 5345-01-12 05:02:41.288456789 5345-01-12 05:02:41.288456789 5344-06-27 07:17:35.041543211 5344-06-27 07:17:35.041543211 5345-01-12 05:02:41.288456789
    +5397-07-13 07:12:32.000896438 5397-04-04 19:49:58.877439649 5397-10-20 18:35:05.124353227 5397-10-20 18:35:05.124353227 5397-04-04 19:49:58.877439649 5397-04-04 19:49:58.877439649 5397-10-20 18:35:05.124353227
    +5966-07-09 03:30:50.597 5966-03-31 16:08:17.473543211 5966-10-16 14:53:23.720456789 5966-10-16 14:53:23.720456789 5966-03-31 16:08:17.473543211 5966-03-31 16:08:17.473543211 5966-10-16 14:53:23.720456789
    +6229-06-28 02:54:28.970117179 6229-03-20 15:31:55.84666039 6229-10-05 14:17:02.093573968 6229-10-05 14:17:02.093573968 6229-03-20 15:31:55.84666039 6229-03-20 15:31:55.84666039 6229-10-05 14:17:02.093573968
    +6482-04-27 12:07:38.073915413 6482-01-17 23:45:04.950458624 6482-08-04 23:30:11.197372202 6482-08-04 23:30:11.197372202 6482-01-17 23:45:04.950458624 6482-01-17 23:45:04.950458624 6482-08-04 23:30:11.197372202
    +6631-11-13 16:31:29.702202248 6631-08-06 06:08:56.578745459 6632-02-21 03:54:02.825659037 6632-02-21 03:54:02.825659037 6631-08-06 06:08:56.578745459 6631-08-06 06:08:56.578745459 6632-02-21 03:54:02.825659037
    +6705-09-28 18:27:28.000845672 6705-06-21 07:04:54.877388883 6706-01-06 04:50:01.124302461 6706-01-06 04:50:01.124302461 6705-06-21 07:04:54.877388883 6705-06-21 07:04:54.877388883 6706-01-06 04:50:01.124302461
    +6731-02-12 08:12:48.287783702 6730-11-04 20:50:15.164326913 6731-05-22 20:35:21.411240491 6731-05-22 20:35:21.411240491 6730-11-04 20:50:15.164326913 6730-11-04 20:50:15.164326913 6731-05-22 20:35:21.411240491
    +7160-12-02 06:00:24.81200852 7160-08-24 19:37:51.688551731 7161-03-11 17:22:57.935465309 7161-03-11 17:22:57.935465309 7160-08-24 19:37:51.688551731 7160-08-24 19:37:51.688551731 7161-03-11 17:22:57.935465309
    +7409-09-07 23:33:32.459349602 7409-05-31 12:10:59.335892813 7409-12-16 09:56:05.582806391 7409-12-16 09:56:05.582806391 7409-05-31 12:10:59.335892813 7409-05-31 12:10:59.335892813 7409-12-16 09:56:05.582806391
    +7503-06-23 23:14:17.486 7503-03-16 11:51:44.362543211 7503-10-01 10:36:50.609456789 7503-10-01 10:36:50.609456789 7503-03-16 11:51:44.362543211 7503-03-16 11:51:44.362543211 7503-10-01 10:36:50.609456789
    +8422-07-22 03:21:45.745036084 8422-04-13 15:59:12.621579295 8422-10-29 14:44:18.868492873 8422-10-29 14:44:18.868492873 8422-04-13 15:59:12.621579295 8422-04-13 15:59:12.621579295 8422-10-29 14:44:18.868492873
    +8521-01-16 20:42:05.668832388 8520-10-09 10:19:32.545375599 8521-04-26 09:04:38.792289177 8521-04-26 09:04:38.792289177 8520-10-09 10:19:32.545375599 8520-10-09 10:19:32.545375599 8521-04-26 09:04:38.792289177
    +9075-06-13 16:20:09.218517797 9075-03-06 03:57:36.095061008 9075-09-21 03:42:42.341974586 9075-09-21 03:42:42.341974586 9075-03-06 03:57:36.095061008 9075-03-06 03:57:36.095061008 9075-09-21 03:42:42.341974586
    +9209-11-11 04:08:58.223768453 9209-08-03 17:46:25.100311664 9210-02-18 15:31:31.347225242 9210-02-18 15:31:31.347225242 9209-08-03 17:46:25.100311664 9209-08-03 17:46:25.100311664 9210-02-18 15:31:31.347225242
    +9403-01-09 18:12:33.547 9402-10-02 07:50:00.423543211 9403-04-19 06:35:06.670456789 9403-04-19 06:35:06.670456789 9402-10-02 07:50:00.423543211 9402-10-02 07:50:00.423543211 9403-04-19 06:35:06.670456789
    +PREHOOK: query: explain
    +select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
    + Select Operator
    + expressions: 109 20:30:40.246913578 (type: interval_day_time), 89 02:14:26.000000000 (type: interval_day_time)
    + outputColumnNames: _col0, _col1
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + Limit
    + Number of rows: 2
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    + Execution mode: vectorized
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: 2
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +_c0 _c1
    +109 20:30:40.246913578 89 02:14:26.000000000
    +109 20:30:40.246913578 89 02:14:26.000000000
    +PREHOOK: query: drop table interval_arithmetic_1
    +PREHOOK: type: DROPTABLE
    +PREHOOK: Input: default@interval_arithmetic_1
    +PREHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: query: drop table interval_arithmetic_1
    +POSTHOOK: type: DROPTABLE
    +POSTHOOK: Input: default@interval_arithmetic_1
    +POSTHOOK: Output: default@interval_arithmetic_1
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13319 : Fix passing external handles in task display (Rajat Khandelwal, reviewed by amareshwari)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/41a30b59
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/41a30b59
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/41a30b59

    Branch: refs/heads/llap
    Commit: 41a30b59de35601211657b65a20b9d418958fb58
    Parents: 69cfd35
    Author: Rajat Khandelwal <prongs@apache.org>
    Authored: Mon Mar 28 09:45:43 2016 +0530
    Committer: Amareshwari Sriramadasu <amareshwari@apache.org>
    Committed: Mon Mar 28 09:45:43 2016 +0530

    ----------------------------------------------------------------------
      ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java | 11 +++++++----
      .../test/org/apache/hive/service/cli/CLIServiceTest.java | 8 ++++++++
      2 files changed, 15 insertions(+), 4 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/41a30b59/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
    index 467dab6..d582bc0 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
    @@ -72,7 +72,7 @@ public class QueryDisplay {
          private Long endTime;

          private String taskId;
    - private String taskExternalHandle;
    + private String externalHandle;

          public Task.TaskState taskState;
          private StageType taskType;
    @@ -85,7 +85,7 @@ public class QueryDisplay {
          }
          public TaskDisplay(Task task) {
            taskId = task.getId();
    - taskExternalHandle = task.getExternalHandle();
    + externalHandle = task.getExternalHandle();
            taskType = task.getType();
            name = task.getName();
            requireLock = task.requireLock();
    @@ -150,12 +150,15 @@ public class QueryDisplay {
          }

          public synchronized String getExternalHandle() {
    - return taskExternalHandle;
    + return externalHandle;
          }

          public synchronized <T extends Serializable> void updateStatus(Task<T> tTask) {
            this.taskState = tTask.getTaskState();
    - switch(taskState) {
    + if (externalHandle == null && tTask.getExternalHandle() != null) {
    + this.externalHandle = tTask.getExternalHandle();
    + }
    + switch (taskState) {
              case RUNNING:
                beginTime = System.currentTimeMillis();
                break;

    http://git-wip-us.apache.org/repos/asf/hive/blob/41a30b59/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
    ----------------------------------------------------------------------
    diff --git a/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java b/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
    index e145eb4..698b13d 100644
    --- a/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
    +++ b/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java
    @@ -45,6 +45,7 @@ import org.codehaus.jackson.type.TypeReference;
      import org.slf4j.Logger;
      import org.slf4j.LoggerFactory;
      import org.apache.hadoop.hive.conf.HiveConf;
    +import org.apache.hadoop.hive.ql.plan.api.StageType;
      import org.apache.hadoop.hive.ql.session.SessionState;

      import org.junit.After;
    @@ -676,6 +677,7 @@ public abstract class CLIServiceTest {
            switch (taskDisplay.taskState) {
              case INITIALIZED:
              case QUEUED:
    + assertNull(taskDisplay.getExternalHandle());
                assertNull(taskDisplay.getBeginTime());
                assertNull(taskDisplay.getEndTime());
                assertNull(taskDisplay.getElapsedTime());
    @@ -683,6 +685,9 @@ public abstract class CLIServiceTest {
                assertNull(taskDisplay.getReturnValue());
                break;
              case RUNNING:
    + if (taskDisplay.getTaskType() == StageType.MAPRED || taskDisplay.getTaskType() == StageType.MAPREDLOCAL) {
    + assertNotNull(taskDisplay.getExternalHandle());
    + }
                assertNotNull(taskDisplay.getBeginTime());
                assertNull(taskDisplay.getEndTime());
                assertNotNull(taskDisplay.getElapsedTime());
    @@ -690,6 +695,9 @@ public abstract class CLIServiceTest {
                assertNull(taskDisplay.getReturnValue());
                break;
              case FINISHED:
    + if (taskDisplay.getTaskType() == StageType.MAPRED || taskDisplay.getTaskType() == StageType.MAPREDLOCAL) {
    + assertNotNull(taskDisplay.getExternalHandle());
    + }
                assertNotNull(taskDisplay.getBeginTime());
                assertNotNull(taskDisplay.getEndTime());
                assertNotNull(taskDisplay.getElapsedTime());
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13358: Stats state is not captured correctly: turn off stats optimizer for sampled table (Pengcheng Xiong, reviewed by Ashutosh Chauhan)


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

    Branch: refs/heads/llap
    Commit: 77474581df4016e3899a986e079513087a945674
    Parents: 41a30b5
    Author: Pengcheng Xiong <pxiong@apache.org>
    Authored: Sun Mar 27 22:30:29 2016 -0700
    Committer: Pengcheng Xiong <pxiong@apache.org>
    Committed: Sun Mar 27 22:30:42 2016 -0700

    ----------------------------------------------------------------------
      .../hive/ql/optimizer/StatsOptimizer.java | 14 +-
      .../sample_islocalmode_hook_use_metadata.q | 48 ++++
      .../sample_islocalmode_hook_use_metadata.q.out | 230 +++++++++++++++++++
      3 files changed, 289 insertions(+), 3 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/77474581/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
    index 4091c0d..bc17fec 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/StatsOptimizer.java
    @@ -100,9 +100,12 @@ public class StatsOptimizer extends Transform {
        @Override
        public ParseContext transform(ParseContext pctx) throws SemanticException {

    - if (pctx.getFetchTask() != null || !pctx.getQueryProperties().isQuery() ||
    - pctx.getQueryProperties().isAnalyzeRewrite() || pctx.getQueryProperties().isCTAS() ||
    - pctx.getLoadFileWork().size() > 1 || !pctx.getLoadTableWork().isEmpty()) {
    + if (pctx.getFetchTask() != null || !pctx.getQueryProperties().isQuery()
    + || pctx.getQueryProperties().isAnalyzeRewrite() || pctx.getQueryProperties().isCTAS()
    + || pctx.getLoadFileWork().size() > 1 || !pctx.getLoadTableWork().isEmpty()
    + // If getNameToSplitSample is not empty, at least one of the source
    + // tables is being sampled and we can not optimize.
    + || !pctx.getNameToSplitSample().isEmpty()) {
            return pctx;
          }

    @@ -251,6 +254,11 @@ public class StatsOptimizer extends Transform {
                // looks like a subq plan.
                return null;
              }
    + if (tsOp.getConf().getRowLimit() != -1) {
    + // table is sampled. In some situation, we really can leverage row
    + // limit. In order to be safe, we do not use it now.
    + return null;
    + }
              SelectOperator pselOp = (SelectOperator)stack.get(1);
              for(ExprNodeDesc desc : pselOp.getConf().getColList()) {
                if (!((desc instanceof ExprNodeColumnDesc) || (desc instanceof ExprNodeConstantDesc))) {

    http://git-wip-us.apache.org/repos/asf/hive/blob/77474581/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q b/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
    new file mode 100644
    index 0000000..ac915b5
    --- /dev/null
    +++ b/ql/src/test/queries/clientpositive/sample_islocalmode_hook_use_metadata.q
    @@ -0,0 +1,48 @@
    +set hive.mapred.mode=nonstrict;
    +set hive.exec.submitviachild=true;
    +set hive.exec.submit.local.task.via.child=true;
    +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
    +set mapred.max.split.size=300;
    +set mapred.min.split.size=300;
    +set mapred.min.split.size.per.node=300;
    +set mapred.min.split.size.per.rack=300;
    +set hive.exec.mode.local.auto=true;
    +set hive.merge.smallfiles.avgsize=1;
    +set hive.compute.query.using.stats=true;
    +
    +-- EXCLUDE_HADOOP_MAJOR_VERSIONS( 0.20S)
    +
    +-- create file inputs
    +create table sih_i_part (key int, value string) partitioned by (p string);
    +insert overwrite table sih_i_part partition (p='1') select key, value from src;
    +insert overwrite table sih_i_part partition (p='2') select key+10000, value from src;
    +insert overwrite table sih_i_part partition (p='3') select key+20000, value from src;
    +create table sih_src as select key, value from sih_i_part order by key, value;
    +create table sih_src2 as select key, value from sih_src order by key, value;
    +
    +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.VerifyIsLocalModeHook;
    +set mapreduce.framework.name=yarn;
    +set mapreduce.jobtracker.address=localhost:58;
    +set hive.sample.seednumber=7;
    +
    +-- Relaxing hive.exec.mode.local.auto.input.files.max=1.
    +-- Hadoop20 will not generate more splits than there are files (one).
    +-- Hadoop23 generate splits correctly (four), hence the max needs to be adjusted to ensure running in local mode.
    +-- Default value is hive.exec.mode.local.auto.input.files.max=4 which produces expected behavior on Hadoop23.
    +-- hive.sample.seednumber is required because Hadoop23 generates multiple splits and tablesample is non-repeatable without it.
    +
    +-- sample split, running locally limited by num tasks
    +
    +desc formatted sih_src;
    +
    +explain select count(1) from sih_src;
    +
    +select count(1) from sih_src;
    +
    +explain select count(1) from sih_src tablesample(1 percent);
    +
    +select count(1) from sih_src tablesample(1 percent);
    +
    +explain select count(1) from sih_src tablesample(10 rows);
    +
    +select count(1) from sih_src tablesample(10 rows);
    \ No newline at end of file

    http://git-wip-us.apache.org/repos/asf/hive/blob/77474581/ql/src/test/results/clientpositive/sample_islocalmode_hook_use_metadata.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/sample_islocalmode_hook_use_metadata.q.out b/ql/src/test/results/clientpositive/sample_islocalmode_hook_use_metadata.q.out
    new file mode 100644
    index 0000000..d268837
    --- /dev/null
    +++ b/ql/src/test/results/clientpositive/sample_islocalmode_hook_use_metadata.q.out
    @@ -0,0 +1,230 @@
    +PREHOOK: query: -- EXCLUDE_HADOOP_MAJOR_VERSIONS( 0.20S)
    +
    +-- create file inputs
    +create table sih_i_part (key int, value string) partitioned by (p string)
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@sih_i_part
    +POSTHOOK: query: -- EXCLUDE_HADOOP_MAJOR_VERSIONS( 0.20S)
    +
    +-- create file inputs
    +create table sih_i_part (key int, value string) partitioned by (p string)
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@sih_i_part
    +PREHOOK: query: insert overwrite table sih_i_part partition (p='1') select key, value from src
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@src
    +PREHOOK: Output: default@sih_i_part@p=1
    +POSTHOOK: query: insert overwrite table sih_i_part partition (p='1') select key, value from src
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@src
    +POSTHOOK: Output: default@sih_i_part@p=1
    +POSTHOOK: Lineage: sih_i_part PARTITION(p=1).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
    +POSTHOOK: Lineage: sih_i_part PARTITION(p=1).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
    +PREHOOK: query: insert overwrite table sih_i_part partition (p='2') select key+10000, value from src
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@src
    +PREHOOK: Output: default@sih_i_part@p=2
    +POSTHOOK: query: insert overwrite table sih_i_part partition (p='2') select key+10000, value from src
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@src
    +POSTHOOK: Output: default@sih_i_part@p=2
    +POSTHOOK: Lineage: sih_i_part PARTITION(p=2).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
    +POSTHOOK: Lineage: sih_i_part PARTITION(p=2).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
    +PREHOOK: query: insert overwrite table sih_i_part partition (p='3') select key+20000, value from src
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@src
    +PREHOOK: Output: default@sih_i_part@p=3
    +POSTHOOK: query: insert overwrite table sih_i_part partition (p='3') select key+20000, value from src
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@src
    +POSTHOOK: Output: default@sih_i_part@p=3
    +POSTHOOK: Lineage: sih_i_part PARTITION(p=3).key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
    +POSTHOOK: Lineage: sih_i_part PARTITION(p=3).value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
    +PREHOOK: query: create table sih_src as select key, value from sih_i_part order by key, value
    +PREHOOK: type: CREATETABLE_AS_SELECT
    +PREHOOK: Input: default@sih_i_part
    +PREHOOK: Input: default@sih_i_part@p=1
    +PREHOOK: Input: default@sih_i_part@p=2
    +PREHOOK: Input: default@sih_i_part@p=3
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@sih_src
    +POSTHOOK: query: create table sih_src as select key, value from sih_i_part order by key, value
    +POSTHOOK: type: CREATETABLE_AS_SELECT
    +POSTHOOK: Input: default@sih_i_part
    +POSTHOOK: Input: default@sih_i_part@p=1
    +POSTHOOK: Input: default@sih_i_part@p=2
    +POSTHOOK: Input: default@sih_i_part@p=3
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@sih_src
    +POSTHOOK: Lineage: sih_src.key SIMPLE [(sih_i_part)sih_i_part.FieldSchema(name:key, type:int, comment:null), ]
    +POSTHOOK: Lineage: sih_src.value SIMPLE [(sih_i_part)sih_i_part.FieldSchema(name:value, type:string, comment:null), ]
    +PREHOOK: query: create table sih_src2 as select key, value from sih_src order by key, value
    +PREHOOK: type: CREATETABLE_AS_SELECT
    +PREHOOK: Input: default@sih_src
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@sih_src2
    +POSTHOOK: query: create table sih_src2 as select key, value from sih_src order by key, value
    +POSTHOOK: type: CREATETABLE_AS_SELECT
    +POSTHOOK: Input: default@sih_src
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@sih_src2
    +POSTHOOK: Lineage: sih_src2.key SIMPLE [(sih_src)sih_src.FieldSchema(name:key, type:int, comment:null), ]
    +POSTHOOK: Lineage: sih_src2.value SIMPLE [(sih_src)sih_src.FieldSchema(name:value, type:string, comment:null), ]
    +PREHOOK: query: -- Relaxing hive.exec.mode.local.auto.input.files.max=1.
    +-- Hadoop20 will not generate more splits than there are files (one).
    +-- Hadoop23 generate splits correctly (four), hence the max needs to be adjusted to ensure running in local mode.
    +-- Default value is hive.exec.mode.local.auto.input.files.max=4 which produces expected behavior on Hadoop23.
    +-- hive.sample.seednumber is required because Hadoop23 generates multiple splits and tablesample is non-repeatable without it.
    +
    +-- sample split, running locally limited by num tasks
    +
    +desc formatted sih_src
    +PREHOOK: type: DESCTABLE
    +PREHOOK: Input: default@sih_src
    +# col_name data_type comment
    +
    +key int
    +value string
    +
    +# Detailed Table Information
    +Database: default
    +#### A masked pattern was here ####
    +Retention: 0
    +#### A masked pattern was here ####
    +Table Type: MANAGED_TABLE
    +Table Parameters:
    + COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
    + numFiles 1
    + numRows 1500
    + rawDataSize 18124
    + totalSize 19624
    +#### A masked pattern was here ####
    +
    +# Storage Information
    +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +InputFormat: org.apache.hadoop.mapred.TextInputFormat
    +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
    +Compressed: No
    +Num Buckets: -1
    +Bucket Columns: []
    +Sort Columns: []
    +Storage Desc Params:
    + serialization.format 1
    +PREHOOK: query: explain select count(1) from sih_src
    +PREHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-0 is a root stage
    +
    +STAGE PLANS:
    + Stage: Stage-0
    + Fetch Operator
    + limit: 1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(1) from sih_src
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@sih_src
    +#### A masked pattern was here ####
    +1500
    +PREHOOK: query: explain select count(1) from sih_src tablesample(1 percent)
    +PREHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: sih_src
    + Statistics: Num rows: 1500 Data size: 18124 Basic stats: COMPLETE Column stats: COMPLETE
    + Select Operator
    + Statistics: Num rows: 1500 Data size: 18124 Basic stats: COMPLETE Column stats: COMPLETE
    + Group By Operator
    + aggregations: count(1)
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + value expressions: _col0 (type: bigint)
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(1) from sih_src tablesample(1 percent)
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@sih_src
    +#### A masked pattern was here ####
    +25
    +PREHOOK: query: explain select count(1) from sih_src tablesample(10 rows)
    +PREHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: sih_src
    + Row Limit Per Split: 10
    + Statistics: Num rows: 1500 Data size: 18124 Basic stats: COMPLETE Column stats: COMPLETE
    + Select Operator
    + Statistics: Num rows: 1500 Data size: 18124 Basic stats: COMPLETE Column stats: COMPLETE
    + Group By Operator
    + aggregations: count(1)
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + value expressions: _col0 (type: bigint)
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(1) from sih_src tablesample(10 rows)
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@sih_src
    +#### A masked pattern was here ####
    +650
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-9499 : hive.limit.query.max.table.partition makes queries fail on non-partitioned tables (Navis via Ashutosh Chauhan)


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

    Branch: refs/heads/llap
    Commit: a71edcf6a5672452a8e00c2bad4f20cffced26d9
    Parents: 3b6b56d
    Author: Navis Ryu <navis@apache.org>
    Authored: Sun Feb 8 17:57:00 2015 -0800
    Committer: Ashutosh Chauhan <hashutosh@apache.org>
    Committed: Thu Mar 24 19:09:47 2016 -0700

    ----------------------------------------------------------------------
      .../ql/optimizer/stats/annotation/StatsRulesProcFactory.java | 3 +--
      .../java/org/apache/hadoop/hive/ql/parse/ParseContext.java | 5 +++++
      .../org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java | 8 ++++++--
      3 files changed, 12 insertions(+), 4 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/a71edcf6/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    index 4bcf6bf..c4fc5ca 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java
    @@ -105,8 +105,7 @@ public class StatsRulesProcFactory {
              Object... nodeOutputs) throws SemanticException {
            TableScanOperator tsop = (TableScanOperator) nd;
            AnnotateStatsProcCtx aspCtx = (AnnotateStatsProcCtx) procCtx;
    - PrunedPartitionList partList =
    - aspCtx.getParseContext().getPrunedPartitions(tsop.getName(), tsop);
    + PrunedPartitionList partList = aspCtx.getParseContext().getPrunedPartitions(tsop);
            Table table = tsop.getConf().getTableMetadata();

            try {

    http://git-wip-us.apache.org/repos/asf/hive/blob/a71edcf6/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
    index 4f784d1..95c254c 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ParseContext.java
    @@ -466,6 +466,11 @@ public class ParseContext {
          this.fetchTask = fetchTask;
        }

    + public PrunedPartitionList getPrunedPartitions(TableScanOperator ts)
    + throws SemanticException {
    + return getPrunedPartitions(ts.getConf().getAlias(), ts);
    + }
    +
        public PrunedPartitionList getPrunedPartitions(String alias, TableScanOperator ts)
            throws SemanticException {
          PrunedPartitionList partsList = opToPartList.get(ts);

    http://git-wip-us.apache.org/repos/asf/hive/blob/a71edcf6/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    index d9db1d5..adee14b 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
    @@ -10776,10 +10776,14 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
              // check whether any of them break the limit
              for (Operator<?> topOp : topOps.values()) {
                if (topOp instanceof TableScanOperator) {
    - if (((TableScanDesc)topOp.getConf()).getIsMetadataOnly()) {
    + TableScanOperator tsOp = (TableScanOperator) topOp;
    + if (tsOp.getConf().getIsMetadataOnly()) {
    + continue;
    + }
    + PrunedPartitionList parts = pCtx.getPrunedPartitions(tsOp);
    + if (!parts.getSourceTable().isPartitioned()) {
                    continue;
                  }
    - PrunedPartitionList parts = pCtx.getOpToPartList().get(topOp);
                  if (parts.getPartitions().size() > scanLimit) {
                    throw new SemanticException(ErrorMsg.PARTITION_SCAN_LIMIT_EXCEEDED, ""
                        + parts.getPartitions().size(), "" + parts.getSourceTable().getTableName(), ""
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12531 : Implement fast-path for Year/Month UDFs for dates between 1999 and 2038 (Jason Dere via Sergey Shelukhin)


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

    Branch: refs/heads/llap
    Commit: e384b2b657c819d5963b8f76222f78bb479a29a2
    Parents: b75d9ea
    Author: Jason Dere <jdere@hortonworks.com>
    Authored: Wed Dec 9 11:48:00 2015 -0800
    Committer: Ashutosh Chauhan <hashutosh@apache.org>
    Committed: Fri Mar 25 07:21:55 2016 -0700

    ----------------------------------------------------------------------
      .../org/apache/hadoop/hive/ql/udf/UDFMonth.java | 16 ++++++++--------
      .../java/org/apache/hadoop/hive/ql/udf/UDFYear.java | 16 ++++++++--------
      .../expressions/TestVectorDateExpressions.java | 13 ++++++++++---
      3 files changed, 26 insertions(+), 19 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/e384b2b6/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMonth.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMonth.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMonth.java
    index 8c2b0e4..05afb8e 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMonth.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMonth.java
    @@ -20,8 +20,8 @@ package org.apache.hadoop.hive.ql.udf;

      import java.text.ParseException;
      import java.text.SimpleDateFormat;
    -import java.util.Calendar;
      import java.util.Date;
    +import org.joda.time.MutableDateTime;

      import org.apache.hadoop.hive.ql.exec.Description;
      import org.apache.hadoop.hive.ql.exec.UDF;
    @@ -53,7 +53,7 @@ import org.apache.hadoop.io.Text;
      @NDV(maxNdv = 31)
      public class UDFMonth extends UDF {
        private final SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd");
    - private final Calendar calendar = Calendar.getInstance();
    + private transient final MutableDateTime mdt = new MutableDateTime();

        private final IntWritable result = new IntWritable();

    @@ -75,8 +75,8 @@ public class UDFMonth extends UDF {
          }
          try {
            Date date = formatter.parse(dateString.toString());
    - calendar.setTime(date);
    - result.set(1 + calendar.get(Calendar.MONTH));
    + mdt.setMillis(date.getTime());
    + result.set(mdt.getMonthOfYear());
            return result;
          } catch (ParseException e) {
            return null;
    @@ -88,8 +88,8 @@ public class UDFMonth extends UDF {
            return null;
          }

    - calendar.setTime(d.get());
    - result.set(1 + calendar.get(Calendar.MONTH));
    + mdt.setMillis(d.get().getTime());
    + result.set(mdt.getMonthOfYear());
          return result;
        }

    @@ -98,8 +98,8 @@ public class UDFMonth extends UDF {
            return null;
          }

    - calendar.setTime(t.getTimestamp());
    - result.set(1 + calendar.get(Calendar.MONTH));
    + mdt.setMillis(t.getTimestamp().getTime());
    + result.set(mdt.getMonthOfYear());
          return result;
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/e384b2b6/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFYear.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFYear.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFYear.java
    index d7ecd8c..fb3a655 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFYear.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFYear.java
    @@ -20,8 +20,8 @@ package org.apache.hadoop.hive.ql.udf;

      import java.text.ParseException;
      import java.text.SimpleDateFormat;
    -import java.util.Calendar;
      import java.util.Date;
    +import org.joda.time.MutableDateTime;

      import org.apache.hadoop.hive.ql.exec.Description;
      import org.apache.hadoop.hive.ql.exec.UDF;
    @@ -53,7 +53,7 @@ import org.apache.hadoop.io.Text;
      @NDV(maxNdv = 20) // although technically its unbounded, its unlikely we will ever see ndv > 20
      public class UDFYear extends UDF {
        private final SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd");
    - private final Calendar calendar = Calendar.getInstance();
    + private transient final MutableDateTime mdt = new MutableDateTime();

        private final IntWritable result = new IntWritable();

    @@ -77,8 +77,8 @@ public class UDFYear extends UDF {

          try {
            Date date = formatter.parse(dateString.toString());
    - calendar.setTime(date);
    - result.set(calendar.get(Calendar.YEAR));
    + mdt.setMillis(date.getTime());
    + result.set(mdt.getYear());
            return result;
          } catch (ParseException e) {
            return null;
    @@ -90,8 +90,8 @@ public class UDFYear extends UDF {
            return null;
          }

    - calendar.setTime(d.get());
    - result.set(calendar.get(Calendar.YEAR));
    + mdt.setMillis(d.get().getTime());
    + result.set(mdt.getYear());
          return result;
        }

    @@ -100,8 +100,8 @@ public class UDFYear extends UDF {
            return null;
          }

    - calendar.setTime(t.getTimestamp());
    - result.set(calendar.get(Calendar.YEAR));
    + mdt.setMillis(t.getTimestamp().getTime());
    + result.set(mdt.getYear());
          return result;
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/e384b2b6/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
    index 58cecc1..61c96e9 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorDateExpressions.java
    @@ -38,7 +38,9 @@ import org.junit.Test;
      import org.junit.internal.runners.statements.Fail;

      import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +import com.sun.tools.javac.resources.javac;

    +import java.sql.Date;
      import java.sql.Timestamp;
      import java.util.ArrayList;
      import java.util.Calendar;
    @@ -53,6 +55,7 @@ import java.util.concurrent.ThreadFactory;
      public class TestVectorDateExpressions {

        private ExecutorService runner;
    + private static final int MAX_SANE_DATE_VALUE = new DateWritable(Date.valueOf("3000-01-01")).getDays();

        /* copied over from VectorUDFTimestampFieldLong */
        private TimestampWritable toTimestampWritable(long daysSinceEpoch) {
    @@ -78,11 +81,15 @@ public class TestVectorDateExpressions {
        }

        private VectorizedRowBatch getVectorizedRandomRowBatch(int seed, int size) {
    + return getVectorizedRandomRowBatch(seed, size, Integer.MAX_VALUE);
    + }
    +
    + private VectorizedRowBatch getVectorizedRandomRowBatch(int seed, int size, int maxValue) {
          VectorizedRowBatch batch = new VectorizedRowBatch(2, size);
          LongColumnVector lcv = new LongColumnVector(size);
          Random rand = new Random(seed);
          for (int i = 0; i < size; i++) {
    - lcv.vector[i] = (rand.nextInt());
    + lcv.vector[i] = (rand.nextInt(maxValue));
          }
          batch.cols[0] = lcv;
          batch.cols[1] = new LongColumnVector(size);
    @@ -159,7 +166,7 @@ public class TestVectorDateExpressions {
          batch.cols[0].isNull[0] = true;
          verifyUDFYear(batch);

    - batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE);
    + batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, MAX_SANE_DATE_VALUE);
          verifyUDFYear(batch);
          TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
          verifyUDFYear(batch);
    @@ -283,7 +290,7 @@ public class TestVectorDateExpressions {
          batch.cols[0].isNull[0] = true;
          verifyUDFMonth(batch);

    - batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE);
    + batch = getVectorizedRandomRowBatch(200, VectorizedRowBatch.DEFAULT_SIZE, MAX_SANE_DATE_VALUE);
          verifyUDFMonth(batch);
          TestVectorizedRowBatch.addRandomNulls(batch.cols[0]);
          verifyUDFMonth(batch);
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out b/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
    index b7b17de..cf6f4c7 100644
    --- a/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
    +++ b/ql/src/test/results/clientpositive/tez/vectorized_casts.q.out
    @@ -353,18 +353,18 @@ true NULL true true true NULL true false true true 11 NULL -64615982 1803053750
      true NULL true true true NULL true false true true 8 NULL 890988972 -1862301000 8 NULL 1 15 NULL NULL 8 8 8 8.0 NULL 8.90988972E8 -1.862301E9 8.0 NULL 1.0 15.892 NULL NULL 8.9098899E8 NULL 1969-12-31 16:00:00.008 NULL 1970-01-10 23:29:48.972 1969-12-10 02:41:39 1969-12-31 16:00:08 NULL 1969-12-31 16:00:00.001 1969-12-31 16:00:00 1969-12-31 16:00:15.892 NULL NULL 8 NULL 890988972 -1862301000 8.0 NULL TRUE 0 1969-12-31 16:00:15.892 XylAH4 XylAH4 XylAH4 8.0 1.781977944E9 0.9893582466233818 8.90988973E8
      true NULL true true true NULL true false true true 8 NULL 930867246 1205399250 8 NULL 1 15 NULL NULL 8 8 8 8.0 NULL 9.30867246E8 1.20539925E9 8.0 NULL 1.0 15.892 NULL NULL 9.3086726E8 NULL 1969-12-31 16:00:00.008 NULL 1970-01-11 10:34:27.246 1970-01-14 14:49:59.25 1969-12-31 16:00:08 NULL 1969-12-31 16:00:00.001 1969-12-31 16:00:00 1969-12-31 16:00:15.892 NULL NULL 8 NULL 930867246 1205399250 8.0 NULL TRUE 0 1969-12-31 16:00:15.892 c1V8o1A c1V8o1A c1V8o1A 8.0 1.861734492E9 0.9893582466233818 9.30867247E8
      true true NULL true true true NULL false true NULL -14 -7196 NULL -1552199500 -14 -7196 NULL 11 NULL NULL -14 -14 -14 -14.0 -7196.0 NULL -1.5521995E9 -14.0 -7196.0 NULL 11.065 NULL NULL NULL -7196.0 1969-12-31 15:59:59.986 1969-12-31 15:59:52.804 NULL 1969-12-13 16:50:00.5 1969-12-31 15:59:46 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:11.065 NULL NULL -14 -7196 NULL -1552199500 -14.0 -7196.0 NULL 0 1969-12-31 16:00:11.065 NULL NULL NULL -14.0 NULL -0.9906073556948704 NULL
    -true true NULL true true true NULL false true NULL -21 -7196 NULL 1542429000 -21 -7196 NULL -4 NULL NULL -21 -21 -21 -21.0 -7196.0 NULL 1.542429E9 -21.0 -7196.0 NULL -4.1 NULL NULL NULL -7196.0 1969-12-31 15:59:59.979 1969-12-31 15:59:52.804 NULL 1970-01-18 12:27:09 1969-12-31 15:59:39 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:55.9 NULL NULL -21 -7196 NULL 1542429000 -21.0 -7196.0 NULL 0 1969-12-31 15:59:55.9 NULL NULL NULL -21.0 NULL -0.8366556385360561 NULL
    -true true NULL true true true NULL false true NULL -24 -7196 NULL 829111000 -24 -7196 NULL -6 NULL NULL -24 -24 -24 -24.0 -7196.0 NULL 8.29111E8 -24.0 -7196.0 NULL -6.855 NULL NULL NULL -7196.0 1969-12-31 15:59:59.976 1969-12-31 15:59:52.804 NULL 1970-01-10 06:18:31 1969-12-31 15:59:36 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.145 NULL NULL -24 -7196 NULL 829111000 -24.0 -7196.0 NULL 0 1969-12-31 15:59:53.145 NULL NULL NULL -24.0 NULL 0.9055783620066238 NULL
    +true true NULL true true true NULL false true NULL -21 -7196 NULL 1542429000 -21 -7196 NULL -5 NULL NULL -21 -21 -21 -21.0 -7196.0 NULL 1.542429E9 -21.0 -7196.0 NULL -4.1 NULL NULL NULL -7196.0 1969-12-31 15:59:59.979 1969-12-31 15:59:52.804 NULL 1970-01-18 12:27:09 1969-12-31 15:59:39 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:55.9 NULL NULL -21 -7196 NULL 1542429000 -21.0 -7196.0 NULL 0 1969-12-31 15:59:55.9 NULL NULL NULL -21.0 NULL -0.8366556385360561 NULL
    +true true NULL true true true NULL false true NULL -24 -7196 NULL 829111000 -24 -7196 NULL -7 NULL NULL -24 -24 -24 -24.0 -7196.0 NULL 8.29111E8 -24.0 -7196.0 NULL -6.855 NULL NULL NULL -7196.0 1969-12-31 15:59:59.976 1969-12-31 15:59:52.804 NULL 1970-01-10 06:18:31 1969-12-31 15:59:36 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.145 NULL NULL -24 -7196 NULL 829111000 -24.0 -7196.0 NULL 0 1969-12-31 15:59:53.145 NULL NULL NULL -24.0 NULL 0.9055783620066238 NULL
      true true NULL true true true NULL false true NULL -30 -200 NULL 1429852250 -30 -200 NULL 12 NULL NULL -30 -30 -30 -30.0 -200.0 NULL 1.42985225E9 -30.0 -200.0 NULL 12.935 NULL NULL NULL -200.0 1969-12-31 15:59:59.97 1969-12-31 15:59:59.8 NULL 1970-01-17 05:10:52.25 1969-12-31 15:59:30 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 16:00:12.935 NULL NULL -30 -200 NULL 1429852250 -30.0 -200.0 NULL 0 1969-12-31 16:00:12.935 NULL NULL NULL -30.0 NULL 0.9880316240928618 NULL
    -true true NULL true true true NULL false true NULL -36 -200 NULL -2006216750 -36 -200 NULL -14 NULL NULL -36 -36 -36 -36.0 -200.0 NULL -2.00621675E9 -36.0 -200.0 NULL -14.252 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1969-12-08 10:43:03.25 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.748 NULL NULL -36 -200 NULL -2006216750 -36.0 -200.0 NULL 0 1969-12-31 15:59:45.748 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    -true true NULL true true true NULL false true NULL -36 -200 NULL 1599879000 -36 -200 NULL -6 NULL NULL -36 -36 -36 -36.0 -200.0 NULL 1.599879E9 -36.0 -200.0 NULL -6.183 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1970-01-19 04:24:39 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.817 NULL NULL -36 -200 NULL 1599879000 -36.0 -200.0 NULL 0 1969-12-31 15:59:53.817 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    -true true NULL true true true NULL false true NULL -38 15601 NULL -1858689000 -38 15601 NULL -1 NULL NULL -38 -38 -38 -38.0 15601.0 NULL -1.858689E9 -38.0 15601.0 NULL -1.386 NULL NULL NULL 15601.0 1969-12-31 15:59:59.962 1969-12-31 16:00:15.601 NULL 1969-12-10 03:41:51 1969-12-31 15:59:22 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:58.614 NULL NULL -38 15601 NULL -1858689000 -38.0 15601.0 NULL 0 1969-12-31 15:59:58.614 NULL NULL NULL -38.0 NULL -0.2963685787093853 NULL
    +true true NULL true true true NULL false true NULL -36 -200 NULL -2006216750 -36 -200 NULL -15 NULL NULL -36 -36 -36 -36.0 -200.0 NULL -2.00621675E9 -36.0 -200.0 NULL -14.252 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1969-12-08 10:43:03.25 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.748 NULL NULL -36 -200 NULL -2006216750 -36.0 -200.0 NULL 0 1969-12-31 15:59:45.748 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    +true true NULL true true true NULL false true NULL -36 -200 NULL 1599879000 -36 -200 NULL -7 NULL NULL -36 -36 -36 -36.0 -200.0 NULL 1.599879E9 -36.0 -200.0 NULL -6.183 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1970-01-19 04:24:39 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.817 NULL NULL -36 -200 NULL 1599879000 -36.0 -200.0 NULL 0 1969-12-31 15:59:53.817 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    +true true NULL true true true NULL false true NULL -38 15601 NULL -1858689000 -38 15601 NULL -2 NULL NULL -38 -38 -38 -38.0 15601.0 NULL -1.858689E9 -38.0 15601.0 NULL -1.3860000000000001 NULL NULL NULL 15601.0 1969-12-31 15:59:59.962 1969-12-31 16:00:15.601 NULL 1969-12-10 03:41:51 1969-12-31 15:59:22 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:58.614 NULL NULL -38 15601 NULL -1858689000 -38.0 15601.0 NULL 0 1969-12-31 15:59:58.614 NULL NULL NULL -38.0 NULL -0.2963685787093853 NULL
      true true NULL true true true NULL false true NULL -5 15601 NULL 612416000 -5 15601 NULL 4 NULL NULL -5 -5 -5 -5.0 15601.0 NULL 6.12416E8 -5.0 15601.0 NULL 4.679 NULL NULL NULL 15601.0 1969-12-31 15:59:59.995 1969-12-31 16:00:15.601 NULL 1970-01-07 18:06:56 1969-12-31 15:59:55 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 16:00:04.679 NULL NULL -5 15601 NULL 612416000 -5.0 15601.0 NULL 0 1969-12-31 16:00:04.679 NULL NULL NULL -5.0 NULL 0.9589242746631385 NULL
    -true true NULL true true true NULL false true NULL -50 -7196 NULL -1031187250 -50 -7196 NULL -5 NULL NULL -50 -50 -50 -50.0 -7196.0 NULL -1.03118725E9 -50.0 -7196.0 NULL -5.267 NULL NULL NULL -7196.0 1969-12-31 15:59:59.95 1969-12-31 15:59:52.804 NULL 1969-12-19 17:33:32.75 1969-12-31 15:59:10 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:54.733 NULL NULL -50 -7196 NULL -1031187250 -50.0 -7196.0 NULL 0 1969-12-31 15:59:54.733 NULL NULL NULL -50.0 NULL 0.26237485370392877 NULL
    +true true NULL true true true NULL false true NULL -50 -7196 NULL -1031187250 -50 -7196 NULL -6 NULL NULL -50 -50 -50 -50.0 -7196.0 NULL -1.03118725E9 -50.0 -7196.0 NULL -5.267 NULL NULL NULL -7196.0 1969-12-31 15:59:59.95 1969-12-31 15:59:52.804 NULL 1969-12-19 17:33:32.75 1969-12-31 15:59:10 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:54.733 NULL NULL -50 -7196 NULL -1031187250 -50.0 -7196.0 NULL 0 1969-12-31 15:59:54.733 NULL NULL NULL -50.0 NULL 0.26237485370392877 NULL
      true true NULL true true true NULL false true NULL -59 -7196 NULL -1604890000 -59 -7196 NULL 13 NULL NULL -59 -59 -59 -59.0 -7196.0 NULL -1.60489E9 -59.0 -7196.0 NULL 13.15 NULL NULL NULL -7196.0 1969-12-31 15:59:59.941 1969-12-31 15:59:52.804 NULL 1969-12-13 02:11:50 1969-12-31 15:59:01 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:13.15 NULL NULL -59 -7196 NULL -1604890000 -59.0 -7196.0 NULL 0 1969-12-31 16:00:13.15 NULL NULL NULL -59.0 NULL -0.6367380071391379 NULL
    -true true NULL true true true NULL false true NULL -60 -7196 NULL 1516314750 -60 -7196 NULL -7 NULL NULL -60 -60 -60 -60.0 -7196.0 NULL 1.51631475E9 -60.0 -7196.0 NULL -7.592 NULL NULL NULL -7196.0 1969-12-31 15:59:59.94 1969-12-31 15:59:52.804 NULL 1970-01-18 05:11:54.75 1969-12-31 15:59:00 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:52.408 NULL NULL -60 -7196 NULL 1516314750 -60.0 -7196.0 NULL 0 1969-12-31 15:59:52.408 NULL NULL NULL -60.0 NULL 0.3048106211022167 NULL
    +true true NULL true true true NULL false true NULL -60 -7196 NULL 1516314750 -60 -7196 NULL -8 NULL NULL -60 -60 -60 -60.0 -7196.0 NULL 1.51631475E9 -60.0 -7196.0 NULL -7.592 NULL NULL NULL -7196.0 1969-12-31 15:59:59.94 1969-12-31 15:59:52.804 NULL 1970-01-18 05:11:54.75 1969-12-31 15:59:00 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:52.408 NULL NULL -60 -7196 NULL 1516314750 -60.0 -7196.0 NULL 0 1969-12-31 15:59:52.408 NULL NULL NULL -60.0 NULL 0.3048106211022167 NULL
      true true NULL true true true NULL false true NULL -8 -7196 NULL -1849991500 -8 -7196 NULL 3 NULL NULL -8 -8 -8 -8.0 -7196.0 NULL -1.8499915E9 -8.0 -7196.0 NULL 3.136 NULL NULL NULL -7196.0 1969-12-31 15:59:59.992 1969-12-31 15:59:52.804 NULL 1969-12-10 06:06:48.5 1969-12-31 15:59:52 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:03.136 NULL NULL -8 -7196 NULL -1849991500 -8.0 -7196.0 NULL 0 1969-12-31 16:00:03.136 NULL NULL NULL -8.0 NULL -0.9893582466233818 NULL
    -true true NULL true true true NULL false true NULL 20 15601 NULL -362433250 20 15601 NULL -14 NULL NULL 20 20 20 20.0 15601.0 NULL -3.6243325E8 20.0 15601.0 NULL -14.871 NULL NULL NULL 15601.0 1969-12-31 16:00:00.02 1969-12-31 16:00:15.601 NULL 1969-12-27 11:19:26.75 1969-12-31 16:00:20 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.129 NULL NULL 20 15601 NULL -362433250 20.0 15601.0 NULL 0 1969-12-31 15:59:45.129 NULL NULL NULL 20.0 NULL 0.9129452507276277 NULL
    -true true NULL true true true NULL false true NULL 48 15601 NULL -795361000 48 15601 NULL -9 NULL NULL 48 48 48 48.0 15601.0 NULL -7.95361E8 48.0 15601.0 NULL -9.765 NULL NULL NULL 15601.0 1969-12-31 16:00:00.048 1969-12-31 16:00:15.601 NULL 1969-12-22 11:03:59 1969-12-31 16:00:48 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:50.235 NULL NULL 48 15601 NULL -795361000 48.0 15601.0 NULL 0 1969-12-31 15:59:50.235 NULL NULL NULL 48.0 NULL -0.7682546613236668 NULL
    +true true NULL true true true NULL false true NULL 20 15601 NULL -362433250 20 15601 NULL -15 NULL NULL 20 20 20 20.0 15601.0 NULL -3.6243325E8 20.0 15601.0 NULL -14.871 NULL NULL NULL 15601.0 1969-12-31 16:00:00.02 1969-12-31 16:00:15.601 NULL 1969-12-27 11:19:26.75 1969-12-31 16:00:20 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.129 NULL NULL 20 15601 NULL -362433250 20.0 15601.0 NULL 0 1969-12-31 15:59:45.129 NULL NULL NULL 20.0 NULL 0.9129452507276277 NULL
    +true true NULL true true true NULL false true NULL 48 15601 NULL -795361000 48 15601 NULL -10 NULL NULL 48 48 48 48.0 15601.0 NULL -7.95361E8 48.0 15601.0 NULL -9.765 NULL NULL NULL 15601.0 1969-12-31 16:00:00.048 1969-12-31 16:00:15.601 NULL 1969-12-22 11:03:59 1969-12-31 16:00:48 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:50.235 NULL NULL 48 15601 NULL -795361000 48.0 15601.0 NULL 0 1969-12-31 15:59:50.235 NULL NULL NULL 48.0 NULL -0.7682546613236668 NULL
      true true NULL true true true NULL false true NULL 5 -7196 NULL -1015607500 5 -7196 NULL 10 NULL NULL 5 5 5 5.0 -7196.0 NULL -1.0156075E9 5.0 -7196.0 NULL 10.973 NULL NULL NULL -7196.0 1969-12-31 16:00:00.005 1969-12-31 15:59:52.804 NULL 1969-12-19 21:53:12.5 1969-12-31 16:00:05 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:10.973 NULL NULL 5 -7196 NULL -1015607500 5.0 -7196.0 NULL 0 1969-12-31 16:00:10.973 NULL NULL NULL 5.0 NULL -0.9589242746631385 NULL
      true true NULL true true true NULL false true NULL 59 -7196 NULL -1137754500 59 -7196 NULL 10 NULL NULL 59 59 59 59.0 -7196.0 NULL -1.1377545E9 59.0 -7196.0 NULL 10.956 NULL NULL NULL -7196.0 1969-12-31 16:00:00.059 1969-12-31 15:59:52.804 NULL 1969-12-18 11:57:25.5 1969-12-31 16:00:59 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:10.956 NULL NULL 59 -7196 NULL -1137754500 59.0 -7196.0 NULL 0 1969-12-31 16:00:10.956 NULL NULL NULL 59.0 NULL 0.6367380071391379 NULL

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/results/clientpositive/tez/vectorized_timestamp.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/tez/vectorized_timestamp.q.out b/ql/src/test/results/clientpositive/tez/vectorized_timestamp.q.out
    new file mode 100644
    index 0000000..5382865
    --- /dev/null
    +++ b/ql/src/test/results/clientpositive/tez/vectorized_timestamp.q.out
    @@ -0,0 +1,157 @@
    +PREHOOK: query: DROP TABLE IF EXISTS test
    +PREHOOK: type: DROPTABLE
    +POSTHOOK: query: DROP TABLE IF EXISTS test
    +POSTHOOK: type: DROPTABLE
    +PREHOOK: query: CREATE TABLE test(ts TIMESTAMP) STORED AS ORC
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@test
    +POSTHOOK: query: CREATE TABLE test(ts TIMESTAMP) STORED AS ORC
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@test
    +PREHOOK: query: INSERT INTO TABLE test VALUES ('0001-01-01 00:00:00.000000000'), ('9999-12-31 23:59:59.999999999')
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@values__tmp__table__1
    +PREHOOK: Output: default@test
    +POSTHOOK: query: INSERT INTO TABLE test VALUES ('0001-01-01 00:00:00.000000000'), ('9999-12-31 23:59:59.999999999')
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@values__tmp__table__1
    +POSTHOOK: Output: default@test
    +POSTHOOK: Lineage: test.ts EXPRESSION [(values__tmp__table__1)values__tmp__table__1.FieldSchema(name:tmp_values_col1, type:string, comment:), ]
    +PREHOOK: query: EXPLAIN
    +SELECT ts FROM test
    +PREHOOK: type: QUERY
    +POSTHOOK: query: EXPLAIN
    +SELECT ts FROM test
    +POSTHOOK: type: QUERY
    +Plan optimized by CBO.
    +
    +Stage-0
    + Fetch Operator
    + limit:-1
    + Stage-1
    + Map 1
    + File Output Operator [FS_2]
    + Select Operator [SEL_1] (rows=2 width=40)
    + Output:["_col0"]
    + TableScan [TS_0] (rows=2 width=40)
    + default@test,test,Tbl:COMPLETE,Col:NONE,Output:["ts"]
    +
    +PREHOOK: query: SELECT ts FROM test
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@test
    +#### A masked pattern was here ####
    +POSTHOOK: query: SELECT ts FROM test
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@test
    +#### A masked pattern was here ####
    +0001-01-01 00:00:00
    +9999-12-31 23:59:59.999999999
    +PREHOOK: query: EXPLAIN
    +SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +PREHOOK: type: QUERY
    +POSTHOOK: query: EXPLAIN
    +SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +POSTHOOK: type: QUERY
    +Plan optimized by CBO.
    +
    +Vertex dependency in root stage
    +Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +
    +Stage-0
    + Fetch Operator
    + limit:-1
    + Stage-1
    + Reducer 2
    + File Output Operator [FS_6]
    + Select Operator [SEL_5] (rows=1 width=80)
    + Output:["_col0","_col1","_col2"]
    + Group By Operator [GBY_4] (rows=1 width=80)
    + Output:["_col0","_col1"],aggregations:["min(VALUE._col0)","max(VALUE._col1)"]
    + <-Map 1 [SIMPLE_EDGE]
    + SHUFFLE [RS_3]
    + Group By Operator [GBY_2] (rows=1 width=80)
    + Output:["_col0","_col1"],aggregations:["min(ts)","max(ts)"]
    + Select Operator [SEL_1] (rows=2 width=40)
    + Output:["ts"]
    + TableScan [TS_0] (rows=2 width=40)
    + default@test,test,Tbl:COMPLETE,Col:NONE,Output:["ts"]
    +
    +PREHOOK: query: SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@test
    +#### A masked pattern was here ####
    +POSTHOOK: query: SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@test
    +#### A masked pattern was here ####
    +0001-01-01 00:00:00 9999-12-31 23:59:59.999999999 3652060 23:59:59.999999999
    +PREHOOK: query: EXPLAIN
    +SELECT ts FROM test
    +PREHOOK: type: QUERY
    +POSTHOOK: query: EXPLAIN
    +SELECT ts FROM test
    +POSTHOOK: type: QUERY
    +Plan optimized by CBO.
    +
    +Stage-0
    + Fetch Operator
    + limit:-1
    + Stage-1
    + Map 1 vectorized
    + File Output Operator [FS_4]
    + Select Operator [OP_3] (rows=2 width=40)
    + Output:["_col0"]
    + TableScan [TS_0] (rows=2 width=40)
    + default@test,test,Tbl:COMPLETE,Col:NONE,Output:["ts"]
    +
    +PREHOOK: query: SELECT ts FROM test
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@test
    +#### A masked pattern was here ####
    +POSTHOOK: query: SELECT ts FROM test
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@test
    +#### A masked pattern was here ####
    +0001-01-01 00:00:00
    +9999-12-31 23:59:59.999999999
    +PREHOOK: query: EXPLAIN
    +SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +PREHOOK: type: QUERY
    +POSTHOOK: query: EXPLAIN
    +SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +POSTHOOK: type: QUERY
    +Plan optimized by CBO.
    +
    +Vertex dependency in root stage
    +Reducer 2 <- Map 1 (SIMPLE_EDGE)
    +
    +Stage-0
    + Fetch Operator
    + limit:-1
    + Stage-1
    + Reducer 2 vectorized
    + File Output Operator [FS_6]
    + Select Operator [SEL_5] (rows=1 width=80)
    + Output:["_col0","_col1","_col2"]
    + Group By Operator [OP_9] (rows=1 width=80)
    + Output:["_col0","_col1"],aggregations:["min(VALUE._col0)","max(VALUE._col1)"]
    + <-Map 1 [SIMPLE_EDGE] vectorized
    + SHUFFLE [RS_3]
    + Group By Operator [OP_8] (rows=1 width=80)
    + Output:["_col0","_col1"],aggregations:["min(ts)","max(ts)"]
    + Select Operator [OP_7] (rows=2 width=40)
    + Output:["ts"]
    + TableScan [TS_0] (rows=2 width=40)
    + default@test,test,Tbl:COMPLETE,Col:NONE,Output:["ts"]
    +
    +PREHOOK: query: SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@test
    +#### A masked pattern was here ####
    +POSTHOOK: query: SELECT MIN(ts), MAX(ts), MAX(ts) - MIN(ts) FROM test
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@test
    +#### A masked pattern was here ####
    +0001-01-01 00:00:00 9999-12-31 23:59:59.999999999 3652060 23:59:59.999999999
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-12992: Hive on tez: Bucket map join plan is incorrect (Vikram Dixit K, reviewed by Jason Dere)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/761b5471
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/761b5471
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/761b5471

    Branch: refs/heads/llap
    Commit: 761b5471a0abbbb38ee35a715ea2d4e6d268d5a9
    Parents: 7747458
    Author: vikram <vikram@hortonworks.com>
    Authored: Mon Mar 28 11:25:11 2016 -0700
    Committer: vikram <vikram@hortonworks.com>
    Committed: Mon Mar 28 11:37:32 2016 -0700

    ----------------------------------------------------------------------
      .../hadoop/hive/ql/exec/OperatorUtils.java | 45 ++-
      .../ql/optimizer/ReduceSinkMapJoinProc.java | 24 +-
      .../clientpositive/bucket_map_join_tez1.q | 27 ++
      .../llap/bucket_map_join_tez1.q.out | 308 +++++++++++++++++++
      .../spark/bucket_map_join_tez1.q.out | 306 ++++++++++++++++++
      .../tez/bucket_map_join_tez1.q.out | 294 ++++++++++++++++++
      6 files changed, 985 insertions(+), 19 deletions(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/761b5471/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
    index 3d664c1..41507b1 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
    @@ -26,6 +26,7 @@ import java.util.Map;
      import java.util.Set;

      import org.apache.hadoop.hive.ql.exec.NodeUtils.Function;
    +import org.apache.hadoop.hive.ql.plan.MapJoinDesc;
      import org.apache.hadoop.hive.ql.plan.OperatorDesc;
      import org.apache.hadoop.mapred.OutputCollector;
      import org.slf4j.Logger;
    @@ -80,6 +81,11 @@ public class OperatorUtils {
          return found.size() == 1 ? found.iterator().next() : null;
        }

    + public static <T> T findSingleOperatorUpstreamJoinAccounted(Operator<?> start, Class<T> clazz) {
    + Set<T> found = findOperatorsUpstreamJoinAccounted(start, clazz, new HashSet<T>());
    + return found.size() == 1 ? found.iterator().next(): null;
    + }
    +
        public static <T> Set<T> findOperatorsUpstream(Collection<Operator<?>> starts, Class<T> clazz) {
          Set<T> found = new HashSet<T>();
          for (Operator<?> start : starts) {
    @@ -101,6 +107,34 @@ public class OperatorUtils {
          return found;
        }

    + public static <T> Set<T> findOperatorsUpstreamJoinAccounted(Operator<?> start, Class<T> clazz,
    + Set<T> found) {
    + if (clazz.isInstance(start)) {
    + found.add((T) start);
    + }
    + int onlyIncludeIndex = -1;
    + if (start instanceof AbstractMapJoinOperator) {
    + AbstractMapJoinOperator mapJoinOp = (AbstractMapJoinOperator) start;
    + MapJoinDesc desc = (MapJoinDesc) mapJoinOp.getConf();
    + onlyIncludeIndex = desc.getPosBigTable();
    + }
    + if (start.getParentOperators() != null) {
    + int i = 0;
    + for (Operator<?> parent : start.getParentOperators()) {
    + if (onlyIncludeIndex >= 0) {
    + if (onlyIncludeIndex == i) {
    + findOperatorsUpstream(parent, clazz, found);
    + }
    + } else {
    + findOperatorsUpstream(parent, clazz, found);
    + }
    + i++;
    + }
    + }
    + return found;
    + }
    +
    +
        public static void setChildrenCollector(List<Operator<? extends OperatorDesc>> childOperators, OutputCollector out) {
          if (childOperators == null) {
            return;
    @@ -202,7 +236,7 @@ public class OperatorUtils {
        }

        public static boolean sameRowSchema(Operator<?> operator1, Operator<?> operator2) {
    - return operator1.getSchema().equals(operator2.getSchema());
    + return operator1.getSchema().equals(operator2.getSchema());
        }

        /**
    @@ -220,9 +254,9 @@ public class OperatorUtils {
         * them
         */
        public static Multimap<Class<? extends Operator<?>>, Operator<?>> classifyOperators(
    - Operator<?> start, Set<Class<? extends Operator<?>>> classes) {
    + Operator<?> start, Set<Class<? extends Operator<?>>> classes) {
          ImmutableMultimap.Builder<Class<? extends Operator<?>>, Operator<?>> resultMap =
    - new ImmutableMultimap.Builder<Class<? extends Operator<?>>, Operator<?>>();
    + new ImmutableMultimap.Builder<Class<? extends Operator<?>>, Operator<?>>();
          List<Operator<?>> ops = new ArrayList<Operator<?>>();
          ops.add(start);
          while (!ops.isEmpty()) {
    @@ -255,9 +289,9 @@ public class OperatorUtils {
         * them
         */
        public static Multimap<Class<? extends Operator<?>>, Operator<?>> classifyOperatorsUpstream(
    - Operator<?> start, Set<Class<? extends Operator<?>>> classes) {
    + Operator<?> start, Set<Class<? extends Operator<?>>> classes) {
          ImmutableMultimap.Builder<Class<? extends Operator<?>>, Operator<?>> resultMap =
    - new ImmutableMultimap.Builder<Class<? extends Operator<?>>, Operator<?>>();
    + new ImmutableMultimap.Builder<Class<? extends Operator<?>>, Operator<?>>();
          List<Operator<?>> ops = new ArrayList<Operator<?>>();
          ops.add(start);
          while (!ops.isEmpty()) {
    @@ -296,5 +330,4 @@ public class OperatorUtils {
          }
          return numberOperators;
        }
    -
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/761b5471/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
    index 1e8f30e..00afc18 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ReduceSinkMapJoinProc.java
    @@ -220,8 +220,8 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
            tableSize = 1;
          }
          LOG.info("Mapjoin " + mapJoinOp + "(bucket map join = )" + joinConf.isBucketMapJoin()
    - + ", pos: " + pos + " --> " + parentWork.getName() + " (" + keyCount
    - + " keys estimated from " + rowCount + " rows, " + bucketCount + " buckets)");
    + + ", pos: " + pos + " --> " + parentWork.getName() + " (" + keyCount
    + + " keys estimated from " + rowCount + " rows, " + bucketCount + " buckets)");
          joinConf.getParentToInput().put(pos, parentWork.getName());
          if (keyCount != Long.MAX_VALUE) {
            joinConf.getParentKeyCounts().put(pos, keyCount);
    @@ -247,10 +247,9 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
             * 4. If we don't find a table scan operator, it has to be a reduce side operation.
             */
            if (mapJoinWork == null) {
    - Operator<?> rootOp =
    - OperatorUtils.findSingleOperatorUpstream(
    - mapJoinOp.getParentOperators().get(joinConf.getPosBigTable()),
    - ReduceSinkOperator.class);
    + Operator<?> rootOp = OperatorUtils.findSingleOperatorUpstreamJoinAccounted(
    + mapJoinOp.getParentOperators().get(joinConf.getPosBigTable()),
    + ReduceSinkOperator.class);
              if (rootOp == null) {
                // likely we found a table scan operator
                edgeType = EdgeType.CUSTOM_EDGE;
    @@ -259,10 +258,9 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
                edgeType = EdgeType.CUSTOM_SIMPLE_EDGE;
              }
            } else {
    - Operator<?> rootOp =
    - OperatorUtils.findSingleOperatorUpstream(
    - mapJoinOp.getParentOperators().get(joinConf.getPosBigTable()),
    - TableScanOperator.class);
    + Operator<?> rootOp = OperatorUtils.findSingleOperatorUpstreamJoinAccounted(
    + mapJoinOp.getParentOperators().get(joinConf.getPosBigTable()),
    + TableScanOperator.class);
              if (rootOp != null) {
                // likely we found a table scan operator
                edgeType = EdgeType.CUSTOM_EDGE;
    @@ -320,7 +318,7 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
          context.linkOpWithWorkMap.put(mapJoinOp, linkWorkMap);

          List<ReduceSinkOperator> reduceSinks
    - = context.linkWorkWithReduceSinkMap.get(parentWork);
    + = context.linkWorkWithReduceSinkMap.get(parentWork);
          if (reduceSinks == null) {
            reduceSinks = new ArrayList<ReduceSinkOperator>();
          }
    @@ -358,7 +356,7 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {
          // let the dummy op be the parent of mapjoin op
          mapJoinOp.replaceParent(parentRS, dummyOp);
          List<Operator<? extends OperatorDesc>> dummyChildren =
    - new ArrayList<Operator<? extends OperatorDesc>>();
    + new ArrayList<Operator<? extends OperatorDesc>>();
          dummyChildren.add(mapJoinOp);
          dummyOp.setChildOperators(dummyChildren);
          dummyOperators.add(dummyOp);
    @@ -384,4 +382,4 @@ public class ReduceSinkMapJoinProc implements NodeProcessor {

          return true;
        }
    -}
    +}
    \ No newline at end of file

    http://git-wip-us.apache.org/repos/asf/hive/blob/761b5471/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
    index 8ed630e..95585db 100644
    --- a/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
    +++ b/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
    @@ -40,6 +40,33 @@ select count(*)
      from
      (select distinct key, value from tab_part) a join tab b on a.key = b.key;

    +explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key;
    +
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key;
    +
    +explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key;
    +
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key;
    +
    +
      -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
      -- In this case the sub-query is chosen as the big table.
      explain

    http://git-wip-us.apache.org/repos/asf/hive/blob/761b5471/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
    index 21cfa5c..204da88 100644
    --- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
    +++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez1.q.out
    @@ -325,6 +325,314 @@ POSTHOOK: Input: default@tab_part
      POSTHOOK: Input: default@tab_part@ds=2008-04-08
      #### A masked pattern was here ####
      242
    +PREHOOK: query: explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +POSTHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Map 2 <- Map 1 (CUSTOM_EDGE), Map 4 (CUSTOM_EDGE)
    + Reducer 3 <- Map 2 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: a
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Execution mode: llap
    + LLAP IO: no inputs
    + Map 2
    + Map Operator Tree:
    + TableScan
    + alias: b
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + outputColumnNames: _col0
    + input vertices:
    + 0 Map 1
    + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + input vertices:
    + 1 Map 4
    + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Group By Operator
    + aggregations: count()
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col0 (type: bigint)
    + Execution mode: llap
    + LLAP IO: no inputs
    + Map 4
    + Map Operator Tree:
    + TableScan
    + alias: b
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Execution mode: llap
    + LLAP IO: no inputs
    + Reducer 3
    + Execution mode: llap
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@tab
    +PREHOOK: Input: default@tab@ds=2008-04-08
    +PREHOOK: Input: default@tab_part
    +PREHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +POSTHOOK: query: select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@tab
    +POSTHOOK: Input: default@tab@ds=2008-04-08
    +POSTHOOK: Input: default@tab_part
    +POSTHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +1166
    +PREHOOK: query: explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +POSTHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Map 3 <- Map 1 (CUSTOM_EDGE), Map 2 (CUSTOM_EDGE)
    + Reducer 4 <- Map 3 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: d
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Execution mode: llap
    + LLAP IO: no inputs
    + Map 2
    + Map Operator Tree:
    + TableScan
    + alias: a
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Execution mode: llap
    + LLAP IO: no inputs
    + Map 3
    + Map Operator Tree:
    + TableScan
    + alias: d
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + outputColumnNames: _col0
    + input vertices:
    + 0 Map 2
    + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + input vertices:
    + 0 Map 1
    + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Group By Operator
    + aggregations: count()
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col0 (type: bigint)
    + Execution mode: llap
    + LLAP IO: no inputs
    + Reducer 4
    + Execution mode: llap
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@tab
    +PREHOOK: Input: default@tab@ds=2008-04-08
    +PREHOOK: Input: default@tab_part
    +PREHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +POSTHOOK: query: select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@tab
    +POSTHOOK: Input: default@tab@ds=2008-04-08
    +POSTHOOK: Input: default@tab_part
    +POSTHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +1166
      PREHOOK: query: -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
      -- In this case the sub-query is chosen as the big table.
      explain

    http://git-wip-us.apache.org/repos/asf/hive/blob/761b5471/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
    index 4899c3a..2d66d35 100644
    --- a/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
    +++ b/ql/src/test/results/clientpositive/spark/bucket_map_join_tez1.q.out
    @@ -326,6 +326,312 @@ POSTHOOK: Input: default@tab_part
      POSTHOOK: Input: default@tab_part@ds=2008-04-08
      #### A masked pattern was here ####
      242
    +PREHOOK: query: explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +POSTHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-2 is a root stage
    + Stage-1 depends on stages: Stage-2
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-2
    + Spark
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: a
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Spark HashTable Sink Operator
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + Local Work:
    + Map Reduce Local Work
    + Map 4
    + Map Operator Tree:
    + TableScan
    + alias: b
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Spark HashTable Sink Operator
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + Local Work:
    + Map Reduce Local Work
    +
    + Stage: Stage-1
    + Spark
    + Edges:
    + Reducer 3 <- Map 2 (GROUP, 1)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 2
    + Map Operator Tree:
    + TableScan
    + alias: b
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + outputColumnNames: _col0
    + input vertices:
    + 0 Map 1
    + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + input vertices:
    + 1 Map 4
    + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
    + Group By Operator
    + aggregations: count()
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col0 (type: bigint)
    + Local Work:
    + Map Reduce Local Work
    + Reducer 3
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@tab
    +PREHOOK: Input: default@tab@ds=2008-04-08
    +PREHOOK: Input: default@tab_part
    +PREHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +POSTHOOK: query: select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@tab
    +POSTHOOK: Input: default@tab@ds=2008-04-08
    +POSTHOOK: Input: default@tab_part
    +POSTHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +1166
    +PREHOOK: query: explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +POSTHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-2 is a root stage
    + Stage-1 depends on stages: Stage-2
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-2
    + Spark
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: d
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Spark HashTable Sink Operator
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + Local Work:
    + Map Reduce Local Work
    + Map 2
    + Map Operator Tree:
    + TableScan
    + alias: a
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Spark HashTable Sink Operator
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + Local Work:
    + Map Reduce Local Work
    +
    + Stage: Stage-1
    + Spark
    + Edges:
    + Reducer 4 <- Map 3 (GROUP, 1)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 3
    + Map Operator Tree:
    + TableScan
    + alias: d
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + outputColumnNames: _col0
    + input vertices:
    + 0 Map 2
    + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + input vertices:
    + 0 Map 1
    + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
    + Group By Operator
    + aggregations: count()
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col0 (type: bigint)
    + Local Work:
    + Map Reduce Local Work
    + Reducer 4
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@tab
    +PREHOOK: Input: default@tab@ds=2008-04-08
    +PREHOOK: Input: default@tab_part
    +PREHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +POSTHOOK: query: select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@tab
    +POSTHOOK: Input: default@tab@ds=2008-04-08
    +POSTHOOK: Input: default@tab_part
    +POSTHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +1166
      PREHOOK: query: -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
      -- In this case the sub-query is chosen as the big table.
      explain

    http://git-wip-us.apache.org/repos/asf/hive/blob/761b5471/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out b/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
    index 2e10157..30c4107 100644
    --- a/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
    +++ b/ql/src/test/results/clientpositive/tez/bucket_map_join_tez1.q.out
    @@ -315,6 +315,300 @@ POSTHOOK: Input: default@tab_part
      POSTHOOK: Input: default@tab_part@ds=2008-04-08
      #### A masked pattern was here ####
      242
    +PREHOOK: query: explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +POSTHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Map 2 <- Map 1 (CUSTOM_EDGE), Map 4 (CUSTOM_EDGE)
    + Reducer 3 <- Map 2 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: a
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Map 2
    + Map Operator Tree:
    + TableScan
    + alias: b
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + outputColumnNames: _col0
    + input vertices:
    + 0 Map 1
    + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + input vertices:
    + 1 Map 4
    + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Group By Operator
    + aggregations: count()
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col0 (type: bigint)
    + Map 4
    + Map Operator Tree:
    + TableScan
    + alias: b
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Reducer 3
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@tab
    +PREHOOK: Input: default@tab@ds=2008-04-08
    +PREHOOK: Input: default@tab_part
    +PREHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +POSTHOOK: query: select count(*)
    +from
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c
    +join
    +tab_part d on c.key = d.key
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@tab
    +POSTHOOK: Input: default@tab@ds=2008-04-08
    +POSTHOOK: Input: default@tab_part
    +POSTHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +1166
    +PREHOOK: query: explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +POSTHOOK: type: QUERY
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Tez
    +#### A masked pattern was here ####
    + Edges:
    + Map 3 <- Map 1 (CUSTOM_EDGE), Map 2 (CUSTOM_EDGE)
    + Reducer 4 <- Map 3 (SIMPLE_EDGE)
    +#### A masked pattern was here ####
    + Vertices:
    + Map 1
    + Map Operator Tree:
    + TableScan
    + alias: d
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map 2
    + Map Operator Tree:
    + TableScan
    + alias: a
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: int)
    + sort order: +
    + Map-reduce partition columns: _col0 (type: int)
    + Statistics: Num rows: 242 Data size: 2566 Basic stats: COMPLETE Column stats: NONE
    + Map 3
    + Map Operator Tree:
    + TableScan
    + alias: d
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Filter Operator
    + predicate: key is not null (type: boolean)
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: key (type: int)
    + outputColumnNames: _col0
    + Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + outputColumnNames: _col0
    + input vertices:
    + 0 Map 2
    + Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Map Join Operator
    + condition map:
    + Inner Join 0 to 1
    + keys:
    + 0 _col0 (type: int)
    + 1 _col0 (type: int)
    + input vertices:
    + 0 Map 1
    + Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
    + HybridGraceHashJoin: true
    + Group By Operator
    + aggregations: count()
    + mode: hash
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + sort order:
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col0 (type: bigint)
    + Reducer 4
    + Reduce Operator Tree:
    + Group By Operator
    + aggregations: count(VALUE._col0)
    + mode: mergepartial
    + outputColumnNames: _col0
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@tab
    +PREHOOK: Input: default@tab@ds=2008-04-08
    +PREHOOK: Input: default@tab_part
    +PREHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +POSTHOOK: query: select count(*)
    +from
    +tab_part d
    +join
    +(select a.key as key, a.value as value from tab a join tab_part b on a.key = b.key) c on c.key = d.key
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@tab
    +POSTHOOK: Input: default@tab@ds=2008-04-08
    +POSTHOOK: Input: default@tab_part
    +POSTHOOK: Input: default@tab_part@ds=2008-04-08
    +#### A masked pattern was here ####
    +1166
      PREHOOK: query: -- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
      -- In this case the sub-query is chosen as the big table.
      explain
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
    new file mode 100644
    index 0000000..f4e55ed
    --- /dev/null
    +++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsExtrapolation.java
    @@ -0,0 +1,717 @@
    +/**
    + * 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.hadoop.hive.metastore.hbase;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.hadoop.hbase.Cell;
    +import org.apache.hadoop.hbase.client.HTableInterface;
    +import org.apache.hadoop.hive.conf.HiveConf;
    +import org.apache.hadoop.hive.metastore.StatObjectConverter;
    +import org.apache.hadoop.hive.metastore.api.AggrStats;
    +import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    +import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.FieldSchema;
    +import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.Partition;
    +import org.apache.hadoop.hive.metastore.api.SerDeInfo;
    +import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
    +import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.Table;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.mockito.Mock;
    +import org.mockito.MockitoAnnotations;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +
    +public class TestHBaseAggregateStatsExtrapolation {
    + private static final Logger LOG = LoggerFactory
    + .getLogger(TestHBaseAggregateStatsExtrapolation.class.getName());
    +
    + @Mock
    + HTableInterface htable;
    + private HBaseStore store;
    + SortedMap<String, Cell> rows = new TreeMap<>();
    +
    + // NDV will be 3 for the bitVectors
    + String bitVectors = "{0, 4, 5, 7}{0, 1}{0, 1, 2}{0, 1, 4}{0}{0, 2}{0, 3}{0, 2, 3, 4}{0, 1, 4}{0, 1}{0}{0, 1, 3, 8}{0, 2}{0, 2}{0, 9}{0, 1, 4}";
    +
    + @Before
    + public void before() throws IOException {
    + MockitoAnnotations.initMocks(this);
    + HiveConf conf = new HiveConf();
    + conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true);
    + store = MockUtils.init(conf, htable, rows);
    + store.backdoor().getStatsCache().resetCounters();
    + }
    +
    + private static interface Checker {
    + void checkStats(AggrStats aggrStats) throws Exception;
    + }
    +
    + @Test
    + public void allPartitionsHaveBitVectorStatusLong() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col1", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col1");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col1", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1009, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1009, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col1"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void allPartitionsHaveBitVectorStatusDecimal() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col1_decimal", "decimal", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col1_decimal");
    + obj.setColType("decimal");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + DecimalColumnStatsData dcsd = new DecimalColumnStatsData();
    + dcsd.setHighValue(StatObjectConverter.createThriftDecimal("" + (1000 + i)));
    + dcsd.setLowValue(StatObjectConverter.createThriftDecimal("" + (-1000 - i)));
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors);
    + data.setDecimalStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col1_decimal", cso.getColName());
    + Assert.assertEquals("decimal", cso.getColType());
    + DecimalColumnStatsData lcsd = cso.getStatsData().getDecimalStats();
    + Assert.assertEquals(1009, HBaseUtils.getDoubleValue(lcsd.getHighValue()), 0.01);
    + Assert.assertEquals(-1009, HBaseUtils.getDoubleValue(lcsd.getLowValue()), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col1_decimal"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void allPartitionsHaveBitVectorStatusDouble() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col1_double", "double", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col1_double");
    + obj.setColType("double");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + DoubleColumnStatsData dcsd = new DoubleColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors);
    + data.setDoubleStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col1_double", cso.getColName());
    + Assert.assertEquals("double", cso.getColType());
    + DoubleColumnStatsData lcsd = cso.getStatsData().getDoubleStats();
    + Assert.assertEquals(1009, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1009, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col1_double"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void allPartitionsHaveBitVectorStatusString() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col1_string", "string", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col1_string");
    + obj.setColType("string");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + StringColumnStatsData dcsd = new StringColumnStatsData();
    + dcsd.setAvgColLen(i + 1);
    + dcsd.setMaxColLen(i + 10);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors);
    + data.setStringStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col1_string", cso.getColName());
    + Assert.assertEquals("string", cso.getColType());
    + StringColumnStatsData lcsd = cso.getStatsData().getStringStats();
    + Assert.assertEquals(10, lcsd.getAvgColLen(), 0.01);
    + Assert.assertEquals(19, lcsd.getMaxColLen(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col1_string"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void noPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col2", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col2");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col2", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1009, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1009, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(90, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col2"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsOfPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col3", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i < 2 || i > 7) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col3");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i);
    + dcsd.setBitVectors(bitVectors);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(4, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col3", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1010, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1010, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col3"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void MiddleOfPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col4", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i > 2 && i < 7) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col4");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i);
    + dcsd.setBitVectors(bitVectors);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(4, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col4", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1006, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1006, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col4"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsAndMiddleOfPartitionsHaveBitVectorStatusLong() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col5", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i == 0 || i == 2 || i == 3 || i == 5 || i == 6 || i == 8) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col5");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i);
    + dcsd.setBitVectors(bitVectors);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(6, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col5", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1010, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1010, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(40, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col5"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsAndMiddleOfPartitionsHaveBitVectorStatusDouble() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col5_double", "double", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i == 0 || i == 2 || i == 3 || i == 5 || i == 6 || i == 8) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col5_double");
    + obj.setColType("double");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + DoubleColumnStatsData dcsd = new DoubleColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i);
    + dcsd.setBitVectors(bitVectors);
    + data.setDoubleStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(6, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col5_double", cso.getColName());
    + Assert.assertEquals("double", cso.getColType());
    + DoubleColumnStatsData lcsd = cso.getStatsData().getDoubleStats();
    + Assert.assertEquals(1010, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1010, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(40, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col5_double"));
    + statChecker.checkStats(aggrStats);
    + }
    +}

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
    ----------------------------------------------------------------------
    diff --git a/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
    new file mode 100644
    index 0000000..62918be
    --- /dev/null
    +++ b/metastore/src/test/org/apache/hadoop/hive/metastore/hbase/TestHBaseAggregateStatsNDVUniformDist.java
    @@ -0,0 +1,581 @@
    +/**
    + * 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.hadoop.hive.metastore.hbase;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.hadoop.hbase.Cell;
    +import org.apache.hadoop.hbase.client.HTableInterface;
    +import org.apache.hadoop.hive.conf.HiveConf;
    +import org.apache.hadoop.hive.metastore.StatObjectConverter;
    +import org.apache.hadoop.hive.metastore.api.AggrStats;
    +import org.apache.hadoop.hive.metastore.api.BooleanColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
    +import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
    +import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.FieldSchema;
    +import org.apache.hadoop.hive.metastore.api.LongColumnStatsData;
    +import org.apache.hadoop.hive.metastore.api.Partition;
    +import org.apache.hadoop.hive.metastore.api.SerDeInfo;
    +import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
    +import org.apache.hadoop.hive.metastore.api.Table;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.mockito.Mock;
    +import org.mockito.MockitoAnnotations;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +
    +public class TestHBaseAggregateStatsNDVUniformDist {
    + private static final Logger LOG = LoggerFactory
    + .getLogger(TestHBaseAggregateStatsNDVUniformDist.class.getName());
    +
    + @Mock
    + HTableInterface htable;
    + private HBaseStore store;
    + SortedMap<String, Cell> rows = new TreeMap<>();
    +
    + // NDV will be 3 for bitVectors[0] and 12 for bitVectors[1]
    + String bitVectors[] = {
    + "{0, 4, 5, 7}{0, 1}{0, 1, 2}{0, 1, 4}{0}{0, 2}{0, 3}{0, 2, 3, 4}{0, 1, 4}{0, 1}{0}{0, 1, 3, 8}{0, 2}{0, 2}{0, 9}{0, 1, 4}",
    + "{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}{1, 2}" };
    +
    + @Before
    + public void before() throws IOException {
    + MockitoAnnotations.initMocks(this);
    + HiveConf conf = new HiveConf();
    + conf.setBoolean(HBaseReadWrite.NO_CACHE_CONF, true);
    + conf.setBoolean(HiveConf.ConfVars.HIVE_METASTORE_STATS_NDV_DENSITY_FUNCTION.varname, true);
    + store = MockUtils.init(conf, htable, rows);
    + store.backdoor().getStatsCache().resetCounters();
    + }
    +
    + private static interface Checker {
    + void checkStats(AggrStats aggrStats) throws Exception;
    + }
    +
    + @Test
    + public void allPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col1", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col1");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors[0]);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col1", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1009, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1009, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col1"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void noPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col2", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col2");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(10, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col2", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1009, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1009, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(91, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col2"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsOfPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col3", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i < 2 || i > 7) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col3");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors[i / 5]);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(4, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col3", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1010, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1010, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(12, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col3"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void MiddleOfPartitionsHaveBitVectorStatus() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col4", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i > 2 && i < 7) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col4");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors[0]);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(4, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col4", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1006, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1006, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(45, lcsd.getNumNulls());
    + Assert.assertEquals(3, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col4"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsAndMiddleOfPartitionsHaveBitVectorStatusLong() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col5_long", "long", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i == 0 || i == 2 || i == 3 || i == 5 || i == 6 || i == 8) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col5_long");
    + obj.setColType("long");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + LongColumnStatsData dcsd = new LongColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors[i / 5]);
    + data.setLongStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(6, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col5_long", cso.getColName());
    + Assert.assertEquals("long", cso.getColType());
    + LongColumnStatsData lcsd = cso.getStatsData().getLongStats();
    + Assert.assertEquals(1010, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1010, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(40, lcsd.getNumNulls());
    + Assert.assertEquals(12, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col5_long"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsAndMiddleOfPartitionsHaveBitVectorStatusDecimal() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col5_decimal", "decimal", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i == 0 || i == 2 || i == 3 || i == 5 || i == 6 || i == 8) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col5_decimal");
    + obj.setColType("decimal");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + DecimalColumnStatsData dcsd = new DecimalColumnStatsData();
    + dcsd.setHighValue(StatObjectConverter.createThriftDecimal("" + (1000 + i)));
    + dcsd.setLowValue(StatObjectConverter.createThriftDecimal("" + (-1000 - i)));
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors[i / 5]);
    + data.setDecimalStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(6, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col5_decimal", cso.getColName());
    + Assert.assertEquals("decimal", cso.getColType());
    + DecimalColumnStatsData lcsd = cso.getStatsData().getDecimalStats();
    + Assert.assertEquals(1010, HBaseUtils.getDoubleValue(lcsd.getHighValue()), 0.01);
    + Assert.assertEquals(-1010, HBaseUtils.getDoubleValue(lcsd.getLowValue()), 0.01);
    + Assert.assertEquals(40, lcsd.getNumNulls());
    + Assert.assertEquals(12, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col5_decimal"));
    + statChecker.checkStats(aggrStats);
    + }
    +
    + @Test
    + public void TwoEndsAndMiddleOfPartitionsHaveBitVectorStatusDouble() throws Exception {
    + String dbName = "default";
    + String tableName = "snp";
    + long now = System.currentTimeMillis();
    + List<FieldSchema> cols = new ArrayList<>();
    + cols.add(new FieldSchema("col5_double", "double", "nocomment"));
    + SerDeInfo serde = new SerDeInfo("serde", "seriallib", null);
    + StorageDescriptor sd = new StorageDescriptor(cols, "file:/tmp", "input", "output", false, 0,
    + serde, null, null, Collections.<String, String> emptyMap());
    + List<FieldSchema> partCols = new ArrayList<>();
    + partCols.add(new FieldSchema("ds", "string", ""));
    + Table table = new Table(tableName, dbName, "me", (int) now, (int) now, 0, sd, partCols,
    + Collections.<String, String> emptyMap(), null, null, null);
    + store.createTable(table);
    +
    + List<List<String>> partVals = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + List<String> partVal = Arrays.asList("" + i);
    + partVals.add(partVal);
    + StorageDescriptor psd = new StorageDescriptor(sd);
    + psd.setLocation("file:/tmp/default/hit/ds=" + partVal);
    + Partition part = new Partition(partVal, dbName, tableName, (int) now, (int) now, psd,
    + Collections.<String, String> emptyMap());
    + store.addPartition(part);
    + if (i == 0 || i == 2 || i == 3 || i == 5 || i == 6 || i == 8) {
    + ColumnStatistics cs = new ColumnStatistics();
    + ColumnStatisticsDesc desc = new ColumnStatisticsDesc(false, dbName, tableName);
    + desc.setLastAnalyzed(now);
    + desc.setPartName("ds=" + partVal);
    + cs.setStatsDesc(desc);
    + ColumnStatisticsObj obj = new ColumnStatisticsObj();
    + obj.setColName("col5_double");
    + obj.setColType("double");
    + ColumnStatisticsData data = new ColumnStatisticsData();
    + DoubleColumnStatsData dcsd = new DoubleColumnStatsData();
    + dcsd.setHighValue(1000 + i);
    + dcsd.setLowValue(-1000 - i);
    + dcsd.setNumNulls(i);
    + dcsd.setNumDVs(10 * i + 1);
    + dcsd.setBitVectors(bitVectors[i / 5]);
    + data.setDoubleStats(dcsd);
    + obj.setStatsData(data);
    + cs.addToStatsObj(obj);
    + store.updatePartitionColumnStatistics(cs, partVal);
    + }
    + }
    +
    + Checker statChecker = new Checker() {
    + @Override
    + public void checkStats(AggrStats aggrStats) throws Exception {
    + Assert.assertEquals(6, aggrStats.getPartsFound());
    + Assert.assertEquals(1, aggrStats.getColStatsSize());
    + ColumnStatisticsObj cso = aggrStats.getColStats().get(0);
    + Assert.assertEquals("col5_double", cso.getColName());
    + Assert.assertEquals("double", cso.getColType());
    + DoubleColumnStatsData lcsd = cso.getStatsData().getDoubleStats();
    + Assert.assertEquals(1010, lcsd.getHighValue(), 0.01);
    + Assert.assertEquals(-1010, lcsd.getLowValue(), 0.01);
    + Assert.assertEquals(40, lcsd.getNumNulls());
    + Assert.assertEquals(12, lcsd.getNumDVs());
    + }
    + };
    + List<String> partNames = new ArrayList<>();
    + for (int i = 0; i < 10; i++) {
    + partNames.add("ds=" + i);
    + }
    + AggrStats aggrStats = store.get_aggr_stats_for(dbName, tableName, partNames,
    + Arrays.asList("col5_double"));
    + statChecker.checkStats(aggrStats);
    + }
    +}

    http://git-wip-us.apache.org/repos/asf/hive/blob/96862093/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
    index b501f97..0eb9132 100644
    --- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
    +++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
    @@ -426,9 +426,9 @@ Stage-0
                                                <-Map 8 [SIMPLE_EDGE]
                                                  SHUFFLE [RS_15]
                                                    PartitionCols:_col0, _col1, _col2
    - Group By Operator [GBY_14] (rows=1 width=101)
    + Group By Operator [GBY_14] (rows=2 width=101)
                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
    - Filter Operator [FIL_49] (rows=3 width=93)
    + Filter Operator [FIL_49] (rows=5 width=74)
                                                        predicate:((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                                                        TableScan [TS_11] (rows=20 width=83)
                                                          default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
    @@ -446,9 +446,9 @@ Stage-0
                                                <-Map 1 [SIMPLE_EDGE]
                                                  SHUFFLE [RS_4]
                                                    PartitionCols:_col0, _col1, _col2
    - Group By Operator [GBY_3] (rows=1 width=101)
    + Group By Operator [GBY_3] (rows=2 width=101)
                                                      Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
    - Filter Operator [FIL_48] (rows=3 width=93)
    + Filter Operator [FIL_48] (rows=5 width=74)
                                                        predicate:((((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                                                        TableScan [TS_0] (rows=20 width=83)
                                                          default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
    @@ -1201,11 +1201,11 @@ Stage-0
          Stage-1
            Reducer 3
            File Output Operator [FS_19]
    - Select Operator [SEL_18] (rows=21 width=101)
    + Select Operator [SEL_18] (rows=36 width=101)
                Output:["_col0","_col1","_col2","_col3","_col4"]
    - Filter Operator [FIL_17] (rows=21 width=101)
    + Filter Operator [FIL_17] (rows=36 width=101)
                  predicate:((_col1 > 0) or (_col6 >= 0))
    - Merge Join Operator [MERGEJOIN_28] (rows=21 width=101)
    + Merge Join Operator [MERGEJOIN_28] (rows=36 width=101)
                    Conds:RS_14._col0=RS_15._col0(Inner),Output:["_col1","_col2","_col3","_col4","_col6"]
                  <-Map 5 [SIMPLE_EDGE]
                    SHUFFLE [RS_15]
    @@ -1219,25 +1219,25 @@ Stage-0
                  <-Reducer 2 [SIMPLE_EDGE]
                    SHUFFLE [RS_14]
                      PartitionCols:_col0
    - Filter Operator [FIL_9] (rows=6 width=182)
    + Filter Operator [FIL_9] (rows=10 width=182)
                        predicate:(((_col1 + _col4) = 2) and ((_col4 + 1) = 2))
    - Merge Join Operator [MERGEJOIN_27] (rows=25 width=182)
    + Merge Join Operator [MERGEJOIN_27] (rows=40 width=182)
                          Conds:RS_6._col0=RS_7._col0(Left Outer),Output:["_col0","_col1","_col2","_col3","_col4"]
                        <-Map 1 [SIMPLE_EDGE]
                          SHUFFLE [RS_6]
                            PartitionCols:_col0
    - Select Operator [SEL_2] (rows=5 width=74)
    + Select Operator [SEL_2] (rows=9 width=82)
                              Output:["_col0","_col1","_col2"]
    - Filter Operator [FIL_24] (rows=5 width=74)
    + Filter Operator [FIL_24] (rows=9 width=82)
                                predicate:((((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                                TableScan [TS_0] (rows=20 width=83)
                                  default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
                        <-Map 4 [SIMPLE_EDGE]
                          SHUFFLE [RS_7]
                            PartitionCols:_col0
    - Select Operator [SEL_5] (rows=5 width=71)
    + Select Operator [SEL_5] (rows=9 width=79)
                              Output:["_col0","_col1"]
    - Filter Operator [FIL_25] (rows=5 width=74)
    + Filter Operator [FIL_25] (rows=9 width=82)
                                predicate:((((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                                TableScan [TS_3] (rows=20 width=83)
                                  default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
    @@ -1257,27 +1257,27 @@ Stage-0
          Stage-1
            Reducer 2
            File Output Operator [FS_14]
    - Select Operator [SEL_13] (rows=24 width=101)
    + Select Operator [SEL_13] (rows=50 width=101)
                Output:["_col0","_col1","_col2","_col3","_col4"]
    - Filter Operator [FIL_12] (rows=24 width=101)
    + Filter Operator [FIL_12] (rows=50 width=101)
                  predicate:(((_col1 + _col4) = 2) and ((_col1 > 0) or (_col6 >= 0)) and ((_col4 + 1) = 2))
    - Merge Join Operator [MERGEJOIN_19] (rows=72 width=101)
    + Merge Join Operator [MERGEJOIN_19] (rows=200 width=101)
                    Conds:RS_8._col0=RS_9._col0(Right Outer),RS_8._col0=RS_10._col0(Right Outer),Output:["_col1","_col2","_col3","_col4","_col6"]
                  <-Map 1 [SIMPLE_EDGE]
                    SHUFFLE [RS_8]
                      PartitionCols:_col0
    - Select Operator [SEL_2] (rows=6 width=77)
    + Select Operator [SEL_2] (rows=10 width=83)
                        Output:["_col0","_col1","_col2"]
    - Filter Operator [FIL_17] (rows=6 width=77)
    + Filter Operator [FIL_17] (rows=10 width=83)
                          predicate:(((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0)))
                          TableScan [TS_0] (rows=20 width=83)
                            default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
                  <-Map 3 [SIMPLE_EDGE]
                    SHUFFLE [RS_9]
                      PartitionCols:_col0
    - Select Operator [SEL_5] (rows=6 width=74)
    + Select Operator [SEL_5] (rows=10 width=80)
                        Output:["_col0","_col1"]
    - Filter Operator [FIL_18] (rows=6 width=77)
    + Filter Operator [FIL_18] (rows=10 width=83)
                          predicate:(((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0)))
                          TableScan [TS_3] (rows=20 width=83)
                            default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
    @@ -1509,53 +1509,53 @@ Stage-0
                                      Output:["_col0","_col1","_col2"]
                                      Filter Operator [FIL_31] (rows=1 width=101)
                                        predicate:((_col1 + _col4) >= 0)
    - Merge Join Operator [MERGEJOIN_60] (rows=1 width=101)
    + Merge Join Operator [MERGEJOIN_60] (rows=2 width=101)
                                          Conds:RS_28._col0=RS_29._col0(Inner),Output:["_col0","_col1","_col2","_col4"]
                                        <-Reducer 10 [SIMPLE_EDGE]
                                          SHUFFLE [RS_29]
                                            PartitionCols:_col0
    - Filter Operator [FIL_26] (rows=1 width=105)
    + Filter Operator [FIL_26] (rows=2 width=62)
                                              predicate:_col0 is not null
    - Limit [LIM_24] (rows=1 width=105)
    + Limit [LIM_24] (rows=3 width=76)
                                                Number of rows:5
    - Select Operator [SEL_23] (rows=1 width=105)
    + Select Operator [SEL_23] (rows=3 width=76)
                                                  Output:["_col0","_col1"]
                                                <-Reducer 9 [SIMPLE_EDGE]
                                                  SHUFFLE [RS_22]
    - Select Operator [SEL_20] (rows=1 width=105)
    + Select Operator [SEL_20] (rows=3 width=76)
                                                      Output:["_col0","_col1","_col2","_col3"]
    - Group By Operator [GBY_19] (rows=1 width=101)
    + Group By Operator [GBY_19] (rows=3 width=70)
                                                        Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                      <-Map 8 [SIMPLE_EDGE]
                                                        SHUFFLE [RS_18]
                                                          PartitionCols:_col0, _col1, _col2
    - Group By Operator [GBY_17] (rows=1 width=101)
    + Group By Operator [GBY_17] (rows=3 width=70)
                                                            Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
    - Filter Operator [FIL_58] (rows=4 width=93)
    + Filter Operator [FIL_58] (rows=6 width=77)
                                                              predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0)))
                                                              TableScan [TS_14] (rows=20 width=83)
                                                                default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
                                        <-Reducer 3 [SIMPLE_EDGE]
                                          SHUFFLE [RS_28]
                                            PartitionCols:_col0
    - Filter Operator [FIL_12] (rows=1 width=97)
    + Filter Operator [FIL_12] (rows=2 width=54)
                                              predicate:_col0 is not null
    - Limit [LIM_10] (rows=1 width=97)
    + Limit [LIM_10] (rows=3 width=68)
                                                Number of rows:5
    - Select Operator [SEL_9] (rows=1 width=97)
    + Select Operator [SEL_9] (rows=3 width=68)
                                                  Output:["_col0","_col1","_col2"]
                                                <-Reducer 2 [SIMPLE_EDGE]
                                                  SHUFFLE [RS_8]
    - Select Operator [SEL_6] (rows=1 width=97)
    + Select Operator [SEL_6] (rows=3 width=68)
                                                      Output:["_col0","_col1","_col2"]
    - Group By Operator [GBY_5] (rows=1 width=101)
    + Group By Operator [GBY_5] (rows=3 width=70)
                                                        Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(VALUE._col0)"],keys:KEY._col0, KEY._col1, KEY._col2
                                                      <-Map 1 [SIMPLE_EDGE]
                                                        SHUFFLE [RS_4]
                                                          PartitionCols:_col0, _col1, _col2
    - Group By Operator [GBY_3] (rows=1 width=101)
    + Group By Operator [GBY_3] (rows=3 width=70)
                                                            Output:["_col0","_col1","_col2","_col3"],aggregations:["sum(c_int)"],keys:key, c_int, c_float
    - Filter Operator [FIL_56] (rows=4 width=93)
    + Filter Operator [FIL_56] (rows=6 width=77)
                                                              predicate:(((c_int + 1) >= 0) and ((c_int > 0) or (c_float >= 0.0)))
                                                              TableScan [TS_0] (rows=20 width=83)
                                                                default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
    @@ -1575,16 +1575,16 @@ Stage-0
          Stage-1
            Reducer 2
            File Output Operator [FS_12]
    - Select Operator [SEL_11] (rows=6 width=4)
    + Select Operator [SEL_11] (rows=11 width=4)
                Output:["_col0"]
    - Merge Join Operator [MERGEJOIN_17] (rows=6 width=4)
    + Merge Join Operator [MERGEJOIN_17] (rows=11 width=4)
                  Conds:RS_8._col0=RS_9._col0(Left Semi),Output:["_col1"]
                <-Map 1 [SIMPLE_EDGE]
                  SHUFFLE [RS_8]
                    PartitionCols:_col0
    - Select Operator [SEL_2] (rows=5 width=74)
    + Select Operator [SEL_2] (rows=9 width=82)
                      Output:["_col0","_col1"]
    - Filter Operator [FIL_15] (rows=5 width=74)
    + Filter Operator [FIL_15] (rows=9 width=82)
                        predicate:((((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                        TableScan [TS_0] (rows=20 width=83)
                          default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
    @@ -1615,27 +1615,27 @@ Stage-0
          Stage-1
            Reducer 2
            File Output Operator [FS_18]
    - Select Operator [SEL_17] (rows=12 width=93)
    + Select Operator [SEL_17] (rows=16 width=93)
                Output:["_col0","_col1","_col2"]
    - Merge Join Operator [MERGEJOIN_28] (rows=12 width=93)
    + Merge Join Operator [MERGEJOIN_28] (rows=16 width=93)
                  Conds:RS_13._col0=RS_14._col0(Left Semi),RS_13._col0=RS_15._col0(Left Semi),Output:["_col0","_col1","_col2"]
                <-Map 1 [SIMPLE_EDGE]
                  SHUFFLE [RS_13]
                    PartitionCols:_col0
    - Select Operator [SEL_2] (rows=5 width=74)
    + Select Operator [SEL_2] (rows=9 width=82)
                      Output:["_col0","_col1","_col2"]
    - Filter Operator [FIL_25] (rows=5 width=74)
    + Filter Operator [FIL_25] (rows=9 width=82)
                        predicate:((((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                        TableScan [TS_0] (rows=20 width=83)
                          default@cbo_t1,cbo_t1,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
                <-Map 3 [SIMPLE_EDGE]
                  SHUFFLE [RS_14]
                    PartitionCols:_col0
    - Group By Operator [GBY_10] (rows=2 width=85)
    + Group By Operator [GBY_10] (rows=3 width=85)
                      Output:["_col0"],keys:_col0
    - Select Operator [SEL_5] (rows=5 width=68)
    + Select Operator [SEL_5] (rows=9 width=75)
                        Output:["_col0"]
    - Filter Operator [FIL_26] (rows=5 width=74)
    + Filter Operator [FIL_26] (rows=9 width=82)
                          predicate:((((c_int + 1) = 2) and ((c_int > 0) or (c_float >= 0.0))) and key is not null)
                          TableScan [TS_3] (rows=20 width=83)
                            default@cbo_t2,cbo_t2,Tbl:COMPLETE,Col:COMPLETE,Output:["key","c_int","c_float"]
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out b/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
    new file mode 100644
    index 0000000..cd8111d
    --- /dev/null
    +++ b/ql/src/test/results/clientpositive/vector_interval_arithmetic.q.out
    @@ -0,0 +1,1027 @@
    +PREHOOK: query: create table unique_timestamps (tsval timestamp) STORED AS TEXTFILE
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@unique_timestamps
    +POSTHOOK: query: create table unique_timestamps (tsval timestamp) STORED AS TEXTFILE
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@unique_timestamps
    +PREHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/timestamps.txt' OVERWRITE INTO TABLE unique_timestamps
    +PREHOOK: type: LOAD
    +#### A masked pattern was here ####
    +PREHOOK: Output: default@unique_timestamps
    +POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/timestamps.txt' OVERWRITE INTO TABLE unique_timestamps
    +POSTHOOK: type: LOAD
    +#### A masked pattern was here ####
    +POSTHOOK: Output: default@unique_timestamps
    +PREHOOK: query: create table interval_arithmetic_1 (dateval date, tsval timestamp) stored as orc
    +PREHOOK: type: CREATETABLE
    +PREHOOK: Output: database:default
    +PREHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: query: create table interval_arithmetic_1 (dateval date, tsval timestamp) stored as orc
    +POSTHOOK: type: CREATETABLE
    +POSTHOOK: Output: database:default
    +POSTHOOK: Output: default@interval_arithmetic_1
    +PREHOOK: query: insert overwrite table interval_arithmetic_1
    + select cast(tsval as date), tsval from unique_timestamps
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@unique_timestamps
    +PREHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: query: insert overwrite table interval_arithmetic_1
    + select cast(tsval as date), tsval from unique_timestamps
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@unique_timestamps
    +POSTHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: Lineage: interval_arithmetic_1.dateval EXPRESSION [(unique_timestamps)unique_timestamps.FieldSchema(name:tsval, type:timestamp, comment:null), ]
    +POSTHOOK: Lineage: interval_arithmetic_1.tsval SIMPLE [(unique_timestamps)unique_timestamps.FieldSchema(name:tsval, type:timestamp, comment:null), ]
    +_c0 tsval
    +PREHOOK: query: -- interval year-month arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: -- interval year-month arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), (dateval - 2-2) (type: date), (dateval - -2-2) (type: date), (dateval + 2-2) (type: date), (dateval + -2-2) (type: date), (-2-2 + dateval) (type: date), (2-2 + dateval) (type: date)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: date), _col2 (type: date), _col3 (type: date), _col4 (type: date), _col5 (type: date), _col6 (type: date)
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: date), VALUE._col1 (type: date), VALUE._col2 (type: date), VALUE._col3 (type: date), VALUE._col4 (type: date), VALUE._col5 (type: date)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + dateval - interval '2-2' year to month,
    + dateval - interval '-2-2' year to month,
    + dateval + interval '2-2' year to month,
    + dateval + interval '-2-2' year to month,
    + - interval '2-2' year to month + dateval,
    + interval '2-2' year to month + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval c1 c2 c3 c4 c5 c6
    +0004-09-22 0002-07-22 0006-11-22 0006-11-22 0002-07-22 0002-07-22 0006-11-22
    +0528-10-27 0526-08-27 0530-12-27 0530-12-27 0526-08-27 0526-08-27 0530-12-27
    +1319-02-02 1316-12-02 1321-04-02 1321-04-02 1316-12-02 1316-12-02 1321-04-02
    +1404-07-23 1402-05-23 1406-09-23 1406-09-23 1402-05-23 1402-05-23 1406-09-23
    +1815-05-06 1813-03-06 1817-07-06 1817-07-06 1813-03-06 1813-03-06 1817-07-06
    +1883-04-17 1881-02-17 1885-06-17 1885-06-17 1881-02-17 1881-02-17 1885-06-17
    +1966-08-16 1964-06-16 1968-10-16 1968-10-16 1964-06-16 1964-06-16 1968-10-16
    +1973-04-17 1971-02-17 1975-06-17 1975-06-17 1971-02-17 1971-02-17 1975-06-17
    +1974-10-04 1972-08-04 1976-12-04 1976-12-04 1972-08-04 1972-08-04 1976-12-04
    +1976-03-03 1974-01-03 1978-05-03 1978-05-03 1974-01-03 1974-01-03 1978-05-03
    +1976-05-06 1974-03-06 1978-07-06 1978-07-06 1974-03-06 1974-03-06 1978-07-06
    +1978-08-05 1976-06-05 1980-10-05 1980-10-05 1976-06-05 1976-06-05 1980-10-05
    +1981-04-25 1979-02-25 1983-06-25 1983-06-25 1979-02-25 1979-02-25 1983-06-25
    +1981-11-15 1979-09-15 1984-01-15 1984-01-15 1979-09-15 1979-09-15 1984-01-15
    +1985-07-20 1983-05-20 1987-09-20 1987-09-20 1983-05-20 1983-05-20 1987-09-20
    +1985-11-18 1983-09-18 1988-01-18 1988-01-18 1983-09-18 1983-09-18 1988-01-18
    +1987-02-21 1984-12-21 1989-04-21 1989-04-21 1984-12-21 1984-12-21 1989-04-21
    +1987-05-28 1985-03-28 1989-07-28 1989-07-28 1985-03-28 1985-03-28 1989-07-28
    +1998-10-16 1996-08-16 2000-12-16 2000-12-16 1996-08-16 1996-08-16 2000-12-16
    +1999-10-03 1997-08-03 2001-12-03 2001-12-03 1997-08-03 1997-08-03 2001-12-03
    +2000-12-18 1998-10-18 2003-02-18 2003-02-18 1998-10-18 1998-10-18 2003-02-18
    +2002-05-10 2000-03-10 2004-07-10 2004-07-10 2000-03-10 2000-03-10 2004-07-10
    +2003-09-23 2001-07-23 2005-11-23 2005-11-23 2001-07-23 2001-07-23 2005-11-23
    +2004-03-07 2002-01-07 2006-05-07 2006-05-07 2002-01-07 2002-01-07 2006-05-07
    +2007-02-09 2004-12-09 2009-04-09 2009-04-09 2004-12-09 2004-12-09 2009-04-09
    +2009-01-21 2006-11-21 2011-03-21 2011-03-21 2006-11-21 2006-11-21 2011-03-21
    +2010-04-08 2008-02-08 2012-06-08 2012-06-08 2008-02-08 2008-02-08 2012-06-08
    +2013-04-07 2011-02-07 2015-06-07 2015-06-07 2011-02-07 2011-02-07 2015-06-07
    +2013-04-10 2011-02-10 2015-06-10 2015-06-10 2011-02-10 2011-02-10 2015-06-10
    +2021-09-24 2019-07-24 2023-11-24 2023-11-24 2019-07-24 2019-07-24 2023-11-24
    +2024-11-11 2022-09-11 2027-01-11 2027-01-11 2022-09-11 2022-09-11 2027-01-11
    +4143-07-08 4141-05-08 4145-09-08 4145-09-08 4141-05-08 4141-05-08 4145-09-08
    +4966-12-04 4964-10-04 4969-02-04 4969-02-04 4964-10-04 4964-10-04 4969-02-04
    +5339-02-01 5336-12-01 5341-04-01 5341-04-01 5336-12-01 5336-12-01 5341-04-01
    +5344-10-04 5342-08-04 5346-12-04 5346-12-04 5342-08-04 5342-08-04 5346-12-04
    +5397-07-13 5395-05-13 5399-09-13 5399-09-13 5395-05-13 5395-05-13 5399-09-13
    +5966-07-09 5964-05-09 5968-09-09 5968-09-09 5964-05-09 5964-05-09 5968-09-09
    +6229-06-28 6227-04-28 6231-08-28 6231-08-28 6227-04-28 6227-04-28 6231-08-28
    +6482-04-27 6480-02-27 6484-06-27 6484-06-27 6480-02-27 6480-02-27 6484-06-27
    +6631-11-13 6629-09-13 6634-01-13 6634-01-13 6629-09-13 6629-09-13 6634-01-13
    +6705-09-28 6703-07-28 6707-11-28 6707-11-28 6703-07-28 6703-07-28 6707-11-28
    +6731-02-12 6728-12-12 6733-04-12 6733-04-12 6728-12-12 6728-12-12 6733-04-12
    +7160-12-02 7158-10-02 7163-02-02 7163-02-02 7158-10-02 7158-10-02 7163-02-02
    +7409-09-07 7407-07-07 7411-11-07 7411-11-07 7407-07-07 7407-07-07 7411-11-07
    +7503-06-23 7501-04-23 7505-08-23 7505-08-23 7501-04-23 7501-04-23 7505-08-23
    +8422-07-22 8420-05-22 8424-09-22 8424-09-22 8420-05-22 8420-05-22 8424-09-22
    +8521-01-16 8518-11-16 8523-03-16 8523-03-16 8518-11-16 8518-11-16 8523-03-16
    +9075-06-13 9073-04-13 9077-08-13 9077-08-13 9073-04-13 9073-04-13 9077-08-13
    +9209-11-11 9207-09-11 9212-01-11 9212-01-11 9207-09-11 9207-09-11 9212-01-11
    +9403-01-09 9400-11-09 9405-03-09 9405-03-09 9400-11-09 9400-11-09 9405-03-09
    +PREHOOK: query: explain
    +select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), (dateval - 1999-06-07) (type: interval_day_time), (1999-06-07 - dateval) (type: interval_day_time), (dateval - dateval) (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: interval_day_time), _col2 (type: interval_day_time), _col3 (type: interval_day_time)
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: interval_day_time), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + dateval - date '1999-06-07',
    + date '1999-06-07' - dateval,
    + dateval - dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval c1 c2 c3
    +0004-09-22 -728552 23:00:00.000000000 728552 23:00:00.000000000 0 00:00:00.000000000
    +0528-10-27 -537126 23:00:00.000000000 537126 23:00:00.000000000 0 00:00:00.000000000
    +1319-02-02 -248481 23:00:00.000000000 248481 23:00:00.000000000 0 00:00:00.000000000
    +1404-07-23 -217263 23:00:00.000000000 217263 23:00:00.000000000 0 00:00:00.000000000
    +1815-05-06 -67236 23:00:00.000000000 67236 23:00:00.000000000 0 00:00:00.000000000
    +1883-04-17 -42418 23:00:00.000000000 42418 23:00:00.000000000 0 00:00:00.000000000
    +1966-08-16 -11983 00:00:00.000000000 11983 00:00:00.000000000 0 00:00:00.000000000
    +1973-04-17 -9546 23:00:00.000000000 9546 23:00:00.000000000 0 00:00:00.000000000
    +1974-10-04 -9012 00:00:00.000000000 9012 00:00:00.000000000 0 00:00:00.000000000
    +1976-03-03 -8495 23:00:00.000000000 8495 23:00:00.000000000 0 00:00:00.000000000
    +1976-05-06 -8432 00:00:00.000000000 8432 00:00:00.000000000 0 00:00:00.000000000
    +1978-08-05 -7611 00:00:00.000000000 7611 00:00:00.000000000 0 00:00:00.000000000
    +1981-04-25 -6616 23:00:00.000000000 6616 23:00:00.000000000 0 00:00:00.000000000
    +1981-11-15 -6412 23:00:00.000000000 6412 23:00:00.000000000 0 00:00:00.000000000
    +1985-07-20 -5070 00:00:00.000000000 5070 00:00:00.000000000 0 00:00:00.000000000
    +1985-11-18 -4948 23:00:00.000000000 4948 23:00:00.000000000 0 00:00:00.000000000
    +1987-02-21 -4488 23:00:00.000000000 4488 23:00:00.000000000 0 00:00:00.000000000
    +1987-05-28 -4393 00:00:00.000000000 4393 00:00:00.000000000 0 00:00:00.000000000
    +1998-10-16 -234 00:00:00.000000000 234 00:00:00.000000000 0 00:00:00.000000000
    +1999-10-03 118 00:00:00.000000000 -118 00:00:00.000000000 0 00:00:00.000000000
    +2000-12-18 560 01:00:00.000000000 -560 01:00:00.000000000 0 00:00:00.000000000
    +2002-05-10 1068 00:00:00.000000000 -1068 00:00:00.000000000 0 00:00:00.000000000
    +2003-09-23 1569 00:00:00.000000000 -1569 00:00:00.000000000 0 00:00:00.000000000
    +2004-03-07 1735 01:00:00.000000000 -1735 01:00:00.000000000 0 00:00:00.000000000
    +2007-02-09 2804 01:00:00.000000000 -2804 01:00:00.000000000 0 00:00:00.000000000
    +2009-01-21 3516 01:00:00.000000000 -3516 01:00:00.000000000 0 00:00:00.000000000
    +2010-04-08 3958 00:00:00.000000000 -3958 00:00:00.000000000 0 00:00:00.000000000
    +2013-04-07 5053 00:00:00.000000000 -5053 00:00:00.000000000 0 00:00:00.000000000
    +2013-04-10 5056 00:00:00.000000000 -5056 00:00:00.000000000 0 00:00:00.000000000
    +2021-09-24 8145 00:00:00.000000000 -8145 00:00:00.000000000 0 00:00:00.000000000
    +2024-11-11 9289 01:00:00.000000000 -9289 01:00:00.000000000 0 00:00:00.000000000
    +4143-07-08 783111 00:00:00.000000000 -783111 00:00:00.000000000 0 00:00:00.000000000
    +4966-12-04 1083855 01:00:00.000000000 -1083855 01:00:00.000000000 0 00:00:00.000000000
    +5339-02-01 1219784 01:00:00.000000000 -1219784 01:00:00.000000000 0 00:00:00.000000000
    +5344-10-04 1221856 00:00:00.000000000 -1221856 00:00:00.000000000 0 00:00:00.000000000
    +5397-07-13 1241131 00:00:00.000000000 -1241131 00:00:00.000000000 0 00:00:00.000000000
    +5966-07-09 1448949 00:00:00.000000000 -1448949 00:00:00.000000000 0 00:00:00.000000000
    +6229-06-28 1544997 00:00:00.000000000 -1544997 00:00:00.000000000 0 00:00:00.000000000
    +6482-04-27 1637342 00:00:00.000000000 -1637342 00:00:00.000000000 0 00:00:00.000000000
    +6631-11-13 1691962 01:00:00.000000000 -1691962 01:00:00.000000000 0 00:00:00.000000000
    +6705-09-28 1718944 00:00:00.000000000 -1718944 00:00:00.000000000 0 00:00:00.000000000
    +6731-02-12 1728212 01:00:00.000000000 -1728212 01:00:00.000000000 0 00:00:00.000000000
    +7160-12-02 1885195 01:00:00.000000000 -1885195 01:00:00.000000000 0 00:00:00.000000000
    +7409-09-07 1976054 00:00:00.000000000 -1976054 00:00:00.000000000 0 00:00:00.000000000
    +7503-06-23 2010310 00:00:00.000000000 -2010310 00:00:00.000000000 0 00:00:00.000000000
    +8422-07-22 2345998 00:00:00.000000000 -2345998 00:00:00.000000000 0 00:00:00.000000000
    +8521-01-16 2381970 01:00:00.000000000 -2381970 01:00:00.000000000 0 00:00:00.000000000
    +9075-06-13 2584462 00:00:00.000000000 -2584462 00:00:00.000000000 0 00:00:00.000000000
    +9209-11-11 2633556 01:00:00.000000000 -2633556 01:00:00.000000000 0 00:00:00.000000000
    +9403-01-09 2704106 01:00:00.000000000 -2704106 01:00:00.000000000 0 00:00:00.000000000
    +PREHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: tsval (type: timestamp), (tsval - 2-2) (type: timestamp), (tsval - -2-2) (type: timestamp), (tsval + 2-2) (type: timestamp), (tsval + -2-2) (type: timestamp), (-2-2 + tsval) (type: timestamp), (2-2 + tsval) (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: timestamp)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp)
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: timestamp), VALUE._col0 (type: timestamp), VALUE._col1 (type: timestamp), VALUE._col2 (type: timestamp), VALUE._col3 (type: timestamp), VALUE._col4 (type: timestamp), VALUE._col5 (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + tsval,
    + tsval - interval '2-2' year to month,
    + tsval - interval '-2-2' year to month,
    + tsval + interval '2-2' year to month,
    + tsval + interval '-2-2' year to month,
    + - interval '2-2' year to month + tsval,
    + interval '2-2' year to month + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +tsval c1 c2 c3 c4 c5 c6
    +0004-09-22 18:26:29.519542222 0002-07-22 18:26:29.519542222 0006-11-22 18:26:29.519542222 0006-11-22 18:26:29.519542222 0002-07-22 18:26:29.519542222 0002-07-22 18:26:29.519542222 0006-11-22 18:26:29.519542222
    +0528-10-27 08:15:18.941718273 0526-08-27 08:15:18.941718273 0530-12-27 08:15:18.941718273 0530-12-27 08:15:18.941718273 0526-08-27 08:15:18.941718273 0526-08-27 08:15:18.941718273 0530-12-27 08:15:18.941718273
    +1319-02-02 16:31:57.778 1316-12-02 16:31:57.778 1321-04-02 16:31:57.778 1321-04-02 16:31:57.778 1316-12-02 16:31:57.778 1316-12-02 16:31:57.778 1321-04-02 16:31:57.778
    +1404-07-23 15:32:16.059185026 1402-05-23 15:32:16.059185026 1406-09-23 15:32:16.059185026 1406-09-23 15:32:16.059185026 1402-05-23 15:32:16.059185026 1402-05-23 15:32:16.059185026 1406-09-23 15:32:16.059185026
    +1815-05-06 00:12:37.543584705 1813-03-06 00:12:37.543584705 1817-07-06 00:12:37.543584705 1817-07-06 00:12:37.543584705 1813-03-06 00:12:37.543584705 1813-03-06 00:12:37.543584705 1817-07-06 00:12:37.543584705
    +1883-04-17 04:14:34.647766229 1881-02-17 04:14:34.647766229 1885-06-17 04:14:34.647766229 1885-06-17 04:14:34.647766229 1881-02-17 04:14:34.647766229 1881-02-17 04:14:34.647766229 1885-06-17 04:14:34.647766229
    +1966-08-16 13:36:50.183618031 1964-06-16 13:36:50.183618031 1968-10-16 13:36:50.183618031 1968-10-16 13:36:50.183618031 1964-06-16 13:36:50.183618031 1964-06-16 13:36:50.183618031 1968-10-16 13:36:50.183618031
    +1973-04-17 06:30:38.596784156 1971-02-17 06:30:38.596784156 1975-06-17 07:30:38.596784156 1975-06-17 07:30:38.596784156 1971-02-17 06:30:38.596784156 1971-02-17 06:30:38.596784156 1975-06-17 07:30:38.596784156
    +1974-10-04 17:21:03.989 1972-08-04 17:21:03.989 1976-12-04 16:21:03.989 1976-12-04 16:21:03.989 1972-08-04 17:21:03.989 1972-08-04 17:21:03.989 1976-12-04 16:21:03.989
    +1976-03-03 04:54:33.000895162 1974-01-03 04:54:33.000895162 1978-05-03 05:54:33.000895162 1978-05-03 05:54:33.000895162 1974-01-03 04:54:33.000895162 1974-01-03 04:54:33.000895162 1978-05-03 05:54:33.000895162
    +1976-05-06 00:42:30.910786948 1974-03-06 00:42:30.910786948 1978-07-06 00:42:30.910786948 1978-07-06 00:42:30.910786948 1974-03-06 00:42:30.910786948 1974-03-06 00:42:30.910786948 1978-07-06 00:42:30.910786948
    +1978-08-05 14:41:05.501 1976-06-05 14:41:05.501 1980-10-05 14:41:05.501 1980-10-05 14:41:05.501 1976-06-05 14:41:05.501 1976-06-05 14:41:05.501 1980-10-05 14:41:05.501
    +1981-04-25 09:01:12.077192689 1979-02-25 09:01:12.077192689 1983-06-25 10:01:12.077192689 1983-06-25 10:01:12.077192689 1979-02-25 09:01:12.077192689 1979-02-25 09:01:12.077192689 1983-06-25 10:01:12.077192689
    +1981-11-15 23:03:10.999338387 1979-09-16 00:03:10.999338387 1984-01-15 23:03:10.999338387 1984-01-15 23:03:10.999338387 1979-09-16 00:03:10.999338387 1979-09-16 00:03:10.999338387 1984-01-15 23:03:10.999338387
    +1985-07-20 09:30:11 1983-05-20 09:30:11 1987-09-20 09:30:11 1987-09-20 09:30:11 1983-05-20 09:30:11 1983-05-20 09:30:11 1987-09-20 09:30:11
    +1985-11-18 16:37:54 1983-09-18 17:37:54 1988-01-18 16:37:54 1988-01-18 16:37:54 1983-09-18 17:37:54 1983-09-18 17:37:54 1988-01-18 16:37:54
    +1987-02-21 19:48:29 1984-12-21 19:48:29 1989-04-21 20:48:29 1989-04-21 20:48:29 1984-12-21 19:48:29 1984-12-21 19:48:29 1989-04-21 20:48:29
    +1987-05-28 13:52:07.900916635 1985-03-28 12:52:07.900916635 1989-07-28 13:52:07.900916635 1989-07-28 13:52:07.900916635 1985-03-28 12:52:07.900916635 1985-03-28 12:52:07.900916635 1989-07-28 13:52:07.900916635
    +1998-10-16 20:05:29.397591987 1996-08-16 20:05:29.397591987 2000-12-16 19:05:29.397591987 2000-12-16 19:05:29.397591987 1996-08-16 20:05:29.397591987 1996-08-16 20:05:29.397591987 2000-12-16 19:05:29.397591987
    +1999-10-03 16:59:10.396903939 1997-08-03 16:59:10.396903939 2001-12-03 15:59:10.396903939 2001-12-03 15:59:10.396903939 1997-08-03 16:59:10.396903939 1997-08-03 16:59:10.396903939 2001-12-03 15:59:10.396903939
    +2000-12-18 08:42:30.000595596 1998-10-18 09:42:30.000595596 2003-02-18 08:42:30.000595596 2003-02-18 08:42:30.000595596 1998-10-18 09:42:30.000595596 1998-10-18 09:42:30.000595596 2003-02-18 08:42:30.000595596
    +2002-05-10 05:29:48.990818073 2000-03-10 04:29:48.990818073 2004-07-10 05:29:48.990818073 2004-07-10 05:29:48.990818073 2000-03-10 04:29:48.990818073 2000-03-10 04:29:48.990818073 2004-07-10 05:29:48.990818073
    +2003-09-23 22:33:17.00003252 2001-07-23 22:33:17.00003252 2005-11-23 21:33:17.00003252 2005-11-23 21:33:17.00003252 2001-07-23 22:33:17.00003252 2001-07-23 22:33:17.00003252 2005-11-23 21:33:17.00003252
    +2004-03-07 20:14:13 2002-01-07 20:14:13 2006-05-07 21:14:13 2006-05-07 21:14:13 2002-01-07 20:14:13 2002-01-07 20:14:13 2006-05-07 21:14:13
    +2007-02-09 05:17:29.368756876 2004-12-09 05:17:29.368756876 2009-04-09 06:17:29.368756876 2009-04-09 06:17:29.368756876 2004-12-09 05:17:29.368756876 2004-12-09 05:17:29.368756876 2009-04-09 06:17:29.368756876
    +2009-01-21 10:49:07.108 2006-11-21 10:49:07.108 2011-03-21 11:49:07.108 2011-03-21 11:49:07.108 2006-11-21 10:49:07.108 2006-11-21 10:49:07.108 2011-03-21 11:49:07.108
    +2010-04-08 02:43:35.861742727 2008-02-08 01:43:35.861742727 2012-06-08 02:43:35.861742727 2012-06-08 02:43:35.861742727 2008-02-08 01:43:35.861742727 2008-02-08 01:43:35.861742727 2012-06-08 02:43:35.861742727
    +2013-04-07 02:44:43.00086821 2011-02-07 01:44:43.00086821 2015-06-07 02:44:43.00086821 2015-06-07 02:44:43.00086821 2011-02-07 01:44:43.00086821 2011-02-07 01:44:43.00086821 2015-06-07 02:44:43.00086821
    +2013-04-10 00:43:46.854731546 2011-02-09 23:43:46.854731546 2015-06-10 00:43:46.854731546 2015-06-10 00:43:46.854731546 2011-02-09 23:43:46.854731546 2011-02-09 23:43:46.854731546 2015-06-10 00:43:46.854731546
    +2021-09-24 03:18:32.413655165 2019-07-24 03:18:32.413655165 2023-11-24 02:18:32.413655165 2023-11-24 02:18:32.413655165 2019-07-24 03:18:32.413655165 2019-07-24 03:18:32.413655165 2023-11-24 02:18:32.413655165
    +2024-11-11 16:42:41.101 2022-09-11 17:42:41.101 2027-01-11 16:42:41.101 2027-01-11 16:42:41.101 2022-09-11 17:42:41.101 2022-09-11 17:42:41.101 2027-01-11 16:42:41.101
    +4143-07-08 10:53:27.252802259 4141-05-08 10:53:27.252802259 4145-09-08 10:53:27.252802259 4145-09-08 10:53:27.252802259 4141-05-08 10:53:27.252802259 4141-05-08 10:53:27.252802259 4145-09-08 10:53:27.252802259
    +4966-12-04 09:30:55.202 4964-10-04 10:30:55.202 4969-02-04 09:30:55.202 4969-02-04 09:30:55.202 4964-10-04 10:30:55.202 4964-10-04 10:30:55.202 4969-02-04 09:30:55.202
    +5339-02-01 14:10:01.085678691 5336-12-01 14:10:01.085678691 5341-04-01 15:10:01.085678691 5341-04-01 15:10:01.085678691 5336-12-01 14:10:01.085678691 5336-12-01 14:10:01.085678691 5341-04-01 15:10:01.085678691
    +5344-10-04 18:40:08.165 5342-08-04 18:40:08.165 5346-12-04 17:40:08.165 5346-12-04 17:40:08.165 5342-08-04 18:40:08.165 5342-08-04 18:40:08.165 5346-12-04 17:40:08.165
    +5397-07-13 07:12:32.000896438 5395-05-13 07:12:32.000896438 5399-09-13 07:12:32.000896438 5399-09-13 07:12:32.000896438 5395-05-13 07:12:32.000896438 5395-05-13 07:12:32.000896438 5399-09-13 07:12:32.000896438
    +5966-07-09 03:30:50.597 5964-05-09 03:30:50.597 5968-09-09 03:30:50.597 5968-09-09 03:30:50.597 5964-05-09 03:30:50.597 5964-05-09 03:30:50.597 5968-09-09 03:30:50.597
    +6229-06-28 02:54:28.970117179 6227-04-28 02:54:28.970117179 6231-08-28 02:54:28.970117179 6231-08-28 02:54:28.970117179 6227-04-28 02:54:28.970117179 6227-04-28 02:54:28.970117179 6231-08-28 02:54:28.970117179
    +6482-04-27 12:07:38.073915413 6480-02-27 11:07:38.073915413 6484-06-27 12:07:38.073915413 6484-06-27 12:07:38.073915413 6480-02-27 11:07:38.073915413 6480-02-27 11:07:38.073915413 6484-06-27 12:07:38.073915413
    +6631-11-13 16:31:29.702202248 6629-09-13 17:31:29.702202248 6634-01-13 16:31:29.702202248 6634-01-13 16:31:29.702202248 6629-09-13 17:31:29.702202248 6629-09-13 17:31:29.702202248 6634-01-13 16:31:29.702202248
    +6705-09-28 18:27:28.000845672 6703-07-28 18:27:28.000845672 6707-11-28 17:27:28.000845672 6707-11-28 17:27:28.000845672 6703-07-28 18:27:28.000845672 6703-07-28 18:27:28.000845672 6707-11-28 17:27:28.000845672
    +6731-02-12 08:12:48.287783702 6728-12-12 08:12:48.287783702 6733-04-12 09:12:48.287783702 6733-04-12 09:12:48.287783702 6728-12-12 08:12:48.287783702 6728-12-12 08:12:48.287783702 6733-04-12 09:12:48.287783702
    +7160-12-02 06:00:24.81200852 7158-10-02 07:00:24.81200852 7163-02-02 06:00:24.81200852 7163-02-02 06:00:24.81200852 7158-10-02 07:00:24.81200852 7158-10-02 07:00:24.81200852 7163-02-02 06:00:24.81200852
    +7409-09-07 23:33:32.459349602 7407-07-07 23:33:32.459349602 7411-11-07 22:33:32.459349602 7411-11-07 22:33:32.459349602 7407-07-07 23:33:32.459349602 7407-07-07 23:33:32.459349602 7411-11-07 22:33:32.459349602
    +7503-06-23 23:14:17.486 7501-04-23 23:14:17.486 7505-08-23 23:14:17.486 7505-08-23 23:14:17.486 7501-04-23 23:14:17.486 7501-04-23 23:14:17.486 7505-08-23 23:14:17.486
    +8422-07-22 03:21:45.745036084 8420-05-22 03:21:45.745036084 8424-09-22 03:21:45.745036084 8424-09-22 03:21:45.745036084 8420-05-22 03:21:45.745036084 8420-05-22 03:21:45.745036084 8424-09-22 03:21:45.745036084
    +8521-01-16 20:42:05.668832388 8518-11-16 20:42:05.668832388 8523-03-16 21:42:05.668832388 8523-03-16 21:42:05.668832388 8518-11-16 20:42:05.668832388 8518-11-16 20:42:05.668832388 8523-03-16 21:42:05.668832388
    +9075-06-13 16:20:09.218517797 9073-04-13 16:20:09.218517797 9077-08-13 16:20:09.218517797 9077-08-13 16:20:09.218517797 9073-04-13 16:20:09.218517797 9073-04-13 16:20:09.218517797 9077-08-13 16:20:09.218517797
    +9209-11-11 04:08:58.223768453 9207-09-11 05:08:58.223768453 9212-01-11 04:08:58.223768453 9212-01-11 04:08:58.223768453 9207-09-11 05:08:58.223768453 9207-09-11 05:08:58.223768453 9212-01-11 04:08:58.223768453
    +9403-01-09 18:12:33.547 9400-11-09 18:12:33.547 9405-03-09 18:12:33.547 9405-03-09 18:12:33.547 9400-11-09 18:12:33.547 9400-11-09 18:12:33.547 9405-03-09 18:12:33.547
    +PREHOOK: query: explain
    +select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
    + Select Operator
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + Reduce Output Operator
    + key expressions: 5-5 (type: interval_year_month)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + TopN Hash Memory Usage: 0.1
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: 5-5 (type: interval_year_month), -1-1 (type: interval_year_month)
    + outputColumnNames: _col0, _col1
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + Limit
    + Number of rows: 2
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: 2
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + interval '2-2' year to month + interval '3-3' year to month,
    + interval '2-2' year to month - interval '3-3' year to month
    +from interval_arithmetic_1
    +order by interval '2-2' year to month + interval '3-3' year to month
    +limit 2
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +c0 c1
    +5-5 -1-1
    +5-5 -1-1
    +PREHOOK: query: -- interval day-time arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: -- interval day-time arithmetic
    +explain
    +select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), (dateval - 99 11:22:33.123456789) (type: timestamp), (dateval - -99 11:22:33.123456789) (type: timestamp), (dateval + 99 11:22:33.123456789) (type: timestamp), (dateval + -99 11:22:33.123456789) (type: timestamp), (-99 11:22:33.123456789 + dateval) (type: timestamp), (99 11:22:33.123456789 + dateval) (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp)
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: timestamp), VALUE._col1 (type: timestamp), VALUE._col2 (type: timestamp), VALUE._col3 (type: timestamp), VALUE._col4 (type: timestamp), VALUE._col5 (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + dateval - interval '99 11:22:33.123456789' day to second,
    + dateval - interval '-99 11:22:33.123456789' day to second,
    + dateval + interval '99 11:22:33.123456789' day to second,
    + dateval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + dateval,
    + interval '99 11:22:33.123456789' day to second + dateval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval _c1 _c2 _c3 _c4 _c5 _c6
    +0004-09-22 0004-06-14 12:37:26.876543211 0004-12-30 11:22:33.123456789 0004-12-30 11:22:33.123456789 0004-06-14 12:37:26.876543211 0004-06-14 12:37:26.876543211 0004-12-30 11:22:33.123456789
    +0528-10-27 0528-07-19 12:37:26.876543211 0529-02-03 11:22:33.123456789 0529-02-03 11:22:33.123456789 0528-07-19 12:37:26.876543211 0528-07-19 12:37:26.876543211 0529-02-03 11:22:33.123456789
    +1319-02-02 1318-10-25 12:37:26.876543211 1319-05-12 11:22:33.123456789 1319-05-12 11:22:33.123456789 1318-10-25 12:37:26.876543211 1318-10-25 12:37:26.876543211 1319-05-12 11:22:33.123456789
    +1404-07-23 1404-04-14 12:37:26.876543211 1404-10-30 11:22:33.123456789 1404-10-30 11:22:33.123456789 1404-04-14 12:37:26.876543211 1404-04-14 12:37:26.876543211 1404-10-30 11:22:33.123456789
    +1815-05-06 1815-01-26 12:37:26.876543211 1815-08-13 11:22:33.123456789 1815-08-13 11:22:33.123456789 1815-01-26 12:37:26.876543211 1815-01-26 12:37:26.876543211 1815-08-13 11:22:33.123456789
    +1883-04-17 1883-01-07 12:37:26.876543211 1883-07-25 11:22:33.123456789 1883-07-25 11:22:33.123456789 1883-01-07 12:37:26.876543211 1883-01-07 12:37:26.876543211 1883-07-25 11:22:33.123456789
    +1966-08-16 1966-05-08 12:37:26.876543211 1966-11-23 10:22:33.123456789 1966-11-23 10:22:33.123456789 1966-05-08 12:37:26.876543211 1966-05-08 12:37:26.876543211 1966-11-23 10:22:33.123456789
    +1973-04-17 1973-01-07 12:37:26.876543211 1973-07-25 12:22:33.123456789 1973-07-25 12:22:33.123456789 1973-01-07 12:37:26.876543211 1973-01-07 12:37:26.876543211 1973-07-25 12:22:33.123456789
    +1974-10-04 1974-06-26 12:37:26.876543211 1975-01-11 10:22:33.123456789 1975-01-11 10:22:33.123456789 1974-06-26 12:37:26.876543211 1974-06-26 12:37:26.876543211 1975-01-11 10:22:33.123456789
    +1976-03-03 1975-11-24 12:37:26.876543211 1976-06-10 12:22:33.123456789 1976-06-10 12:22:33.123456789 1975-11-24 12:37:26.876543211 1975-11-24 12:37:26.876543211 1976-06-10 12:22:33.123456789
    +1976-05-06 1976-01-27 11:37:26.876543211 1976-08-13 11:22:33.123456789 1976-08-13 11:22:33.123456789 1976-01-27 11:37:26.876543211 1976-01-27 11:37:26.876543211 1976-08-13 11:22:33.123456789
    +1978-08-05 1978-04-27 11:37:26.876543211 1978-11-12 10:22:33.123456789 1978-11-12 10:22:33.123456789 1978-04-27 11:37:26.876543211 1978-04-27 11:37:26.876543211 1978-11-12 10:22:33.123456789
    +1981-04-25 1981-01-15 12:37:26.876543211 1981-08-02 12:22:33.123456789 1981-08-02 12:22:33.123456789 1981-01-15 12:37:26.876543211 1981-01-15 12:37:26.876543211 1981-08-02 12:22:33.123456789
    +1981-11-15 1981-08-07 13:37:26.876543211 1982-02-22 11:22:33.123456789 1982-02-22 11:22:33.123456789 1981-08-07 13:37:26.876543211 1981-08-07 13:37:26.876543211 1982-02-22 11:22:33.123456789
    +1985-07-20 1985-04-11 11:37:26.876543211 1985-10-27 10:22:33.123456789 1985-10-27 10:22:33.123456789 1985-04-11 11:37:26.876543211 1985-04-11 11:37:26.876543211 1985-10-27 10:22:33.123456789
    +1985-11-18 1985-08-10 13:37:26.876543211 1986-02-25 11:22:33.123456789 1986-02-25 11:22:33.123456789 1985-08-10 13:37:26.876543211 1985-08-10 13:37:26.876543211 1986-02-25 11:22:33.123456789
    +1987-02-21 1986-11-13 12:37:26.876543211 1987-05-31 12:22:33.123456789 1987-05-31 12:22:33.123456789 1986-11-13 12:37:26.876543211 1986-11-13 12:37:26.876543211 1987-05-31 12:22:33.123456789
    +1987-05-28 1987-02-17 11:37:26.876543211 1987-09-04 11:22:33.123456789 1987-09-04 11:22:33.123456789 1987-02-17 11:37:26.876543211 1987-02-17 11:37:26.876543211 1987-09-04 11:22:33.123456789
    +1998-10-16 1998-07-08 12:37:26.876543211 1999-01-23 10:22:33.123456789 1999-01-23 10:22:33.123456789 1998-07-08 12:37:26.876543211 1998-07-08 12:37:26.876543211 1999-01-23 10:22:33.123456789
    +1999-10-03 1999-06-25 12:37:26.876543211 2000-01-10 10:22:33.123456789 2000-01-10 10:22:33.123456789 1999-06-25 12:37:26.876543211 1999-06-25 12:37:26.876543211 2000-01-10 10:22:33.123456789
    +2000-12-18 2000-09-09 13:37:26.876543211 2001-03-27 11:22:33.123456789 2001-03-27 11:22:33.123456789 2000-09-09 13:37:26.876543211 2000-09-09 13:37:26.876543211 2001-03-27 11:22:33.123456789
    +2002-05-10 2002-01-30 11:37:26.876543211 2002-08-17 11:22:33.123456789 2002-08-17 11:22:33.123456789 2002-01-30 11:37:26.876543211 2002-01-30 11:37:26.876543211 2002-08-17 11:22:33.123456789
    +2003-09-23 2003-06-15 12:37:26.876543211 2003-12-31 10:22:33.123456789 2003-12-31 10:22:33.123456789 2003-06-15 12:37:26.876543211 2003-06-15 12:37:26.876543211 2003-12-31 10:22:33.123456789
    +2004-03-07 2003-11-28 12:37:26.876543211 2004-06-14 12:22:33.123456789 2004-06-14 12:22:33.123456789 2003-11-28 12:37:26.876543211 2003-11-28 12:37:26.876543211 2004-06-14 12:22:33.123456789
    +2007-02-09 2006-11-01 12:37:26.876543211 2007-05-19 12:22:33.123456789 2007-05-19 12:22:33.123456789 2006-11-01 12:37:26.876543211 2006-11-01 12:37:26.876543211 2007-05-19 12:22:33.123456789
    +2009-01-21 2008-10-13 13:37:26.876543211 2009-04-30 12:22:33.123456789 2009-04-30 12:22:33.123456789 2008-10-13 13:37:26.876543211 2008-10-13 13:37:26.876543211 2009-04-30 12:22:33.123456789
    +2010-04-08 2009-12-29 11:37:26.876543211 2010-07-16 11:22:33.123456789 2010-07-16 11:22:33.123456789 2009-12-29 11:37:26.876543211 2009-12-29 11:37:26.876543211 2010-07-16 11:22:33.123456789
    +2013-04-07 2012-12-28 11:37:26.876543211 2013-07-15 11:22:33.123456789 2013-07-15 11:22:33.123456789 2012-12-28 11:37:26.876543211 2012-12-28 11:37:26.876543211 2013-07-15 11:22:33.123456789
    +2013-04-10 2012-12-31 11:37:26.876543211 2013-07-18 11:22:33.123456789 2013-07-18 11:22:33.123456789 2012-12-31 11:37:26.876543211 2012-12-31 11:37:26.876543211 2013-07-18 11:22:33.123456789
    +2021-09-24 2021-06-16 12:37:26.876543211 2022-01-01 10:22:33.123456789 2022-01-01 10:22:33.123456789 2021-06-16 12:37:26.876543211 2021-06-16 12:37:26.876543211 2022-01-01 10:22:33.123456789
    +2024-11-11 2024-08-03 13:37:26.876543211 2025-02-18 11:22:33.123456789 2025-02-18 11:22:33.123456789 2024-08-03 13:37:26.876543211 2024-08-03 13:37:26.876543211 2025-02-18 11:22:33.123456789
    +4143-07-08 4143-03-30 12:37:26.876543211 4143-10-15 11:22:33.123456789 4143-10-15 11:22:33.123456789 4143-03-30 12:37:26.876543211 4143-03-30 12:37:26.876543211 4143-10-15 11:22:33.123456789
    +4966-12-04 4966-08-26 13:37:26.876543211 4967-03-13 12:22:33.123456789 4967-03-13 12:22:33.123456789 4966-08-26 13:37:26.876543211 4966-08-26 13:37:26.876543211 4967-03-13 12:22:33.123456789
    +5339-02-01 5338-10-24 13:37:26.876543211 5339-05-11 12:22:33.123456789 5339-05-11 12:22:33.123456789 5338-10-24 13:37:26.876543211 5338-10-24 13:37:26.876543211 5339-05-11 12:22:33.123456789
    +5344-10-04 5344-06-26 12:37:26.876543211 5345-01-11 10:22:33.123456789 5345-01-11 10:22:33.123456789 5344-06-26 12:37:26.876543211 5344-06-26 12:37:26.876543211 5345-01-11 10:22:33.123456789
    +5397-07-13 5397-04-04 12:37:26.876543211 5397-10-20 11:22:33.123456789 5397-10-20 11:22:33.123456789 5397-04-04 12:37:26.876543211 5397-04-04 12:37:26.876543211 5397-10-20 11:22:33.123456789
    +5966-07-09 5966-03-31 12:37:26.876543211 5966-10-16 11:22:33.123456789 5966-10-16 11:22:33.123456789 5966-03-31 12:37:26.876543211 5966-03-31 12:37:26.876543211 5966-10-16 11:22:33.123456789
    +6229-06-28 6229-03-20 12:37:26.876543211 6229-10-05 11:22:33.123456789 6229-10-05 11:22:33.123456789 6229-03-20 12:37:26.876543211 6229-03-20 12:37:26.876543211 6229-10-05 11:22:33.123456789
    +6482-04-27 6482-01-17 11:37:26.876543211 6482-08-04 11:22:33.123456789 6482-08-04 11:22:33.123456789 6482-01-17 11:37:26.876543211 6482-01-17 11:37:26.876543211 6482-08-04 11:22:33.123456789
    +6631-11-13 6631-08-05 13:37:26.876543211 6632-02-20 11:22:33.123456789 6632-02-20 11:22:33.123456789 6631-08-05 13:37:26.876543211 6631-08-05 13:37:26.876543211 6632-02-20 11:22:33.123456789
    +6705-09-28 6705-06-20 12:37:26.876543211 6706-01-05 10:22:33.123456789 6706-01-05 10:22:33.123456789 6705-06-20 12:37:26.876543211 6705-06-20 12:37:26.876543211 6706-01-05 10:22:33.123456789
    +6731-02-12 6730-11-04 12:37:26.876543211 6731-05-22 12:22:33.123456789 6731-05-22 12:22:33.123456789 6730-11-04 12:37:26.876543211 6730-11-04 12:37:26.876543211 6731-05-22 12:22:33.123456789
    +7160-12-02 7160-08-24 13:37:26.876543211 7161-03-11 11:22:33.123456789 7161-03-11 11:22:33.123456789 7160-08-24 13:37:26.876543211 7160-08-24 13:37:26.876543211 7161-03-11 11:22:33.123456789
    +7409-09-07 7409-05-30 12:37:26.876543211 7409-12-15 10:22:33.123456789 7409-12-15 10:22:33.123456789 7409-05-30 12:37:26.876543211 7409-05-30 12:37:26.876543211 7409-12-15 10:22:33.123456789
    +7503-06-23 7503-03-15 12:37:26.876543211 7503-09-30 11:22:33.123456789 7503-09-30 11:22:33.123456789 7503-03-15 12:37:26.876543211 7503-03-15 12:37:26.876543211 7503-09-30 11:22:33.123456789
    +8422-07-22 8422-04-13 12:37:26.876543211 8422-10-29 11:22:33.123456789 8422-10-29 11:22:33.123456789 8422-04-13 12:37:26.876543211 8422-04-13 12:37:26.876543211 8422-10-29 11:22:33.123456789
    +8521-01-16 8520-10-08 13:37:26.876543211 8521-04-25 12:22:33.123456789 8521-04-25 12:22:33.123456789 8520-10-08 13:37:26.876543211 8520-10-08 13:37:26.876543211 8521-04-25 12:22:33.123456789
    +9075-06-13 9075-03-05 11:37:26.876543211 9075-09-20 11:22:33.123456789 9075-09-20 11:22:33.123456789 9075-03-05 11:37:26.876543211 9075-03-05 11:37:26.876543211 9075-09-20 11:22:33.123456789
    +9209-11-11 9209-08-03 13:37:26.876543211 9210-02-18 11:22:33.123456789 9210-02-18 11:22:33.123456789 9209-08-03 13:37:26.876543211 9209-08-03 13:37:26.876543211 9210-02-18 11:22:33.123456789
    +9403-01-09 9402-10-01 13:37:26.876543211 9403-04-18 12:22:33.123456789 9403-04-18 12:22:33.123456789 9402-10-01 13:37:26.876543211 9402-10-01 13:37:26.876543211 9403-04-18 12:22:33.123456789
    +PREHOOK: query: explain
    +select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: dateval (type: date), tsval (type: timestamp), (dateval - tsval) (type: interval_day_time), (tsval - dateval) (type: interval_day_time), (tsval - tsval) (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: date)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: interval_day_time), _col3 (type: interval_day_time), _col4 (type: interval_day_time)
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: date), VALUE._col0 (type: timestamp), VALUE._col1 (type: interval_day_time), VALUE._col2 (type: interval_day_time), VALUE._col3 (type: interval_day_time)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + dateval,
    + tsval,
    + dateval - tsval,
    + tsval - dateval,
    + tsval - tsval
    +from interval_arithmetic_1
    +order by dateval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +dateval tsval c2 c3 c4
    +0004-09-22 0004-09-22 18:26:29.519542222 -0 18:26:30.519542222 0 18:26:30.519542222 0 00:00:00.000000000
    +0528-10-27 0528-10-27 08:15:18.941718273 -0 08:15:19.941718273 0 08:15:19.941718273 0 00:00:00.000000000
    +1319-02-02 1319-02-02 16:31:57.778 -0 16:31:58.778000000 0 16:31:58.778000000 0 00:00:00.000000000
    +1404-07-23 1404-07-23 15:32:16.059185026 -0 15:32:17.059185026 0 15:32:17.059185026 0 00:00:00.000000000
    +1815-05-06 1815-05-06 00:12:37.543584705 -0 00:12:38.543584705 0 00:12:38.543584705 0 00:00:00.000000000
    +1883-04-17 1883-04-17 04:14:34.647766229 -0 04:14:35.647766229 0 04:14:35.647766229 0 00:00:00.000000000
    +1966-08-16 1966-08-16 13:36:50.183618031 -0 13:36:51.183618031 0 13:36:51.183618031 0 00:00:00.000000000
    +1973-04-17 1973-04-17 06:30:38.596784156 -0 06:30:38.596784156 0 06:30:38.596784156 0 00:00:00.000000000
    +1974-10-04 1974-10-04 17:21:03.989 -0 17:21:03.989000000 0 17:21:03.989000000 0 00:00:00.000000000
    +1976-03-03 1976-03-03 04:54:33.000895162 -0 04:54:33.000895162 0 04:54:33.000895162 0 00:00:00.000000000
    +1976-05-06 1976-05-06 00:42:30.910786948 -0 00:42:30.910786948 0 00:42:30.910786948 0 00:00:00.000000000
    +1978-08-05 1978-08-05 14:41:05.501 -0 14:41:05.501000000 0 14:41:05.501000000 0 00:00:00.000000000
    +1981-04-25 1981-04-25 09:01:12.077192689 -0 09:01:12.077192689 0 09:01:12.077192689 0 00:00:00.000000000
    +1981-11-15 1981-11-15 23:03:10.999338387 -0 23:03:10.999338387 0 23:03:10.999338387 0 00:00:00.000000000
    +1985-07-20 1985-07-20 09:30:11 -0 09:30:11.000000000 0 09:30:11.000000000 0 00:00:00.000000000
    +1985-11-18 1985-11-18 16:37:54 -0 16:37:54.000000000 0 16:37:54.000000000 0 00:00:00.000000000
    +1987-02-21 1987-02-21 19:48:29 -0 19:48:29.000000000 0 19:48:29.000000000 0 00:00:00.000000000
    +1987-05-28 1987-05-28 13:52:07.900916635 -0 13:52:07.900916635 0 13:52:07.900916635 0 00:00:00.000000000
    +1998-10-16 1998-10-16 20:05:29.397591987 -0 20:05:29.397591987 0 20:05:29.397591987 0 00:00:00.000000000
    +1999-10-03 1999-10-03 16:59:10.396903939 -0 16:59:10.396903939 0 16:59:10.396903939 0 00:00:00.000000000
    +2000-12-18 2000-12-18 08:42:30.000595596 -0 08:42:30.000595596 0 08:42:30.000595596 0 00:00:00.000000000
    +2002-05-10 2002-05-10 05:29:48.990818073 -0 05:29:48.990818073 0 05:29:48.990818073 0 00:00:00.000000000
    +2003-09-23 2003-09-23 22:33:17.00003252 -0 22:33:17.000032520 0 22:33:17.000032520 0 00:00:00.000000000
    +2004-03-07 2004-03-07 20:14:13 -0 20:14:13.000000000 0 20:14:13.000000000 0 00:00:00.000000000
    +2007-02-09 2007-02-09 05:17:29.368756876 -0 05:17:29.368756876 0 05:17:29.368756876 0 00:00:00.000000000
    +2009-01-21 2009-01-21 10:49:07.108 -0 10:49:07.108000000 0 10:49:07.108000000 0 00:00:00.000000000
    +2010-04-08 2010-04-08 02:43:35.861742727 -0 02:43:35.861742727 0 02:43:35.861742727 0 00:00:00.000000000
    +2013-04-07 2013-04-07 02:44:43.00086821 -0 02:44:43.000868210 0 02:44:43.000868210 0 00:00:00.000000000
    +2013-04-10 2013-04-10 00:43:46.854731546 -0 00:43:46.854731546 0 00:43:46.854731546 0 00:00:00.000000000
    +2021-09-24 2021-09-24 03:18:32.413655165 -0 03:18:32.413655165 0 03:18:32.413655165 0 00:00:00.000000000
    +2024-11-11 2024-11-11 16:42:41.101 -0 16:42:41.101000000 0 16:42:41.101000000 0 00:00:00.000000000
    +4143-07-08 4143-07-08 10:53:27.252802259 -0 10:53:27.252802259 0 10:53:27.252802259 0 00:00:00.000000000
    +4966-12-04 4966-12-04 09:30:55.202 -0 09:30:55.202000000 0 09:30:55.202000000 0 00:00:00.000000000
    +5339-02-01 5339-02-01 14:10:01.085678691 -0 14:10:01.085678691 0 14:10:01.085678691 0 00:00:00.000000000
    +5344-10-04 5344-10-04 18:40:08.165 -0 18:40:08.165000000 0 18:40:08.165000000 0 00:00:00.000000000
    +5397-07-13 5397-07-13 07:12:32.000896438 -0 07:12:32.000896438 0 07:12:32.000896438 0 00:00:00.000000000
    +5966-07-09 5966-07-09 03:30:50.597 -0 03:30:50.597000000 0 03:30:50.597000000 0 00:00:00.000000000
    +6229-06-28 6229-06-28 02:54:28.970117179 -0 02:54:28.970117179 0 02:54:28.970117179 0 00:00:00.000000000
    +6482-04-27 6482-04-27 12:07:38.073915413 -0 12:07:38.073915413 0 12:07:38.073915413 0 00:00:00.000000000
    +6631-11-13 6631-11-13 16:31:29.702202248 -0 16:31:29.702202248 0 16:31:29.702202248 0 00:00:00.000000000
    +6705-09-28 6705-09-28 18:27:28.000845672 -0 18:27:28.000845672 0 18:27:28.000845672 0 00:00:00.000000000
    +6731-02-12 6731-02-12 08:12:48.287783702 -0 08:12:48.287783702 0 08:12:48.287783702 0 00:00:00.000000000
    +7160-12-02 7160-12-02 06:00:24.81200852 -0 06:00:24.812008520 0 06:00:24.812008520 0 00:00:00.000000000
    +7409-09-07 7409-09-07 23:33:32.459349602 -0 23:33:32.459349602 0 23:33:32.459349602 0 00:00:00.000000000
    +7503-06-23 7503-06-23 23:14:17.486 -0 23:14:17.486000000 0 23:14:17.486000000 0 00:00:00.000000000
    +8422-07-22 8422-07-22 03:21:45.745036084 -0 03:21:45.745036084 0 03:21:45.745036084 0 00:00:00.000000000
    +8521-01-16 8521-01-16 20:42:05.668832388 -0 20:42:05.668832388 0 20:42:05.668832388 0 00:00:00.000000000
    +9075-06-13 9075-06-13 16:20:09.218517797 -0 16:20:09.218517797 0 16:20:09.218517797 0 00:00:00.000000000
    +9209-11-11 9209-11-11 04:08:58.223768453 -0 04:08:58.223768453 0 04:08:58.223768453 0 00:00:00.000000000
    +9403-01-09 9403-01-09 18:12:33.547 -0 18:12:33.547000000 0 18:12:33.547000000 0 00:00:00.000000000
    +PREHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Select Operator
    + expressions: tsval (type: timestamp), (tsval - 99 11:22:33.123456789) (type: timestamp), (tsval - -99 11:22:33.123456789) (type: timestamp), (tsval + 99 11:22:33.123456789) (type: timestamp), (tsval + -99 11:22:33.123456789) (type: timestamp), (-99 11:22:33.123456789 + tsval) (type: timestamp), (99 11:22:33.123456789 + tsval) (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + Reduce Output Operator
    + key expressions: _col0 (type: timestamp)
    + sort order: +
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + value expressions: _col1 (type: timestamp), _col2 (type: timestamp), _col3 (type: timestamp), _col4 (type: timestamp), _col5 (type: timestamp), _col6 (type: timestamp)
    + Execution mode: vectorized
    + Reduce Operator Tree:
    + Select Operator
    + expressions: KEY.reducesinkkey0 (type: timestamp), VALUE._col0 (type: timestamp), VALUE._col1 (type: timestamp), VALUE._col2 (type: timestamp), VALUE._col3 (type: timestamp), VALUE._col4 (type: timestamp), VALUE._col5 (type: timestamp)
    + outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: NONE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: -1
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + tsval,
    + tsval - interval '99 11:22:33.123456789' day to second,
    + tsval - interval '-99 11:22:33.123456789' day to second,
    + tsval + interval '99 11:22:33.123456789' day to second,
    + tsval + interval '-99 11:22:33.123456789' day to second,
    + -interval '99 11:22:33.123456789' day to second + tsval,
    + interval '99 11:22:33.123456789' day to second + tsval
    +from interval_arithmetic_1
    +order by tsval
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +tsval _c1 _c2 _c3 _c4 _c5 _c6
    +0004-09-22 18:26:29.519542222 0004-06-15 07:03:56.396085433 0004-12-31 05:49:02.642999011 0004-12-31 05:49:02.642999011 0004-06-15 07:03:56.396085433 0004-06-15 07:03:56.396085433 0004-12-31 05:49:02.642999011
    +0528-10-27 08:15:18.941718273 0528-07-19 20:52:45.818261484 0529-02-03 19:37:52.065175062 0529-02-03 19:37:52.065175062 0528-07-19 20:52:45.818261484 0528-07-19 20:52:45.818261484 0529-02-03 19:37:52.065175062
    +1319-02-02 16:31:57.778 1318-10-26 05:09:24.654543211 1319-05-13 03:54:30.901456789 1319-05-13 03:54:30.901456789 1318-10-26 05:09:24.654543211 1318-10-26 05:09:24.654543211 1319-05-13 03:54:30.901456789
    +1404-07-23 15:32:16.059185026 1404-04-15 04:09:42.935728237 1404-10-31 02:54:49.182641815 1404-10-31 02:54:49.182641815 1404-04-15 04:09:42.935728237 1404-04-15 04:09:42.935728237 1404-10-31 02:54:49.182641815
    +1815-05-06 00:12:37.543584705 1815-01-26 12:50:04.420127916 1815-08-13 11:35:10.667041494 1815-08-13 11:35:10.667041494 1815-01-26 12:50:04.420127916 1815-01-26 12:50:04.420127916 1815-08-13 11:35:10.667041494
    +1883-04-17 04:14:34.647766229 1883-01-07 16:52:01.52430944 1883-07-25 15:37:07.771223018 1883-07-25 15:37:07.771223018 1883-01-07 16:52:01.52430944 1883-01-07 16:52:01.52430944 1883-07-25 15:37:07.771223018
    +1966-08-16 13:36:50.183618031 1966-05-09 02:14:17.060161242 1966-11-23 23:59:23.30707482 1966-11-23 23:59:23.30707482 1966-05-09 02:14:17.060161242 1966-05-09 02:14:17.060161242 1966-11-23 23:59:23.30707482
    +1973-04-17 06:30:38.596784156 1973-01-07 19:08:05.473327367 1973-07-25 18:53:11.720240945 1973-07-25 18:53:11.720240945 1973-01-07 19:08:05.473327367 1973-01-07 19:08:05.473327367 1973-07-25 18:53:11.720240945
    +1974-10-04 17:21:03.989 1974-06-27 05:58:30.865543211 1975-01-12 03:43:37.112456789 1975-01-12 03:43:37.112456789 1974-06-27 05:58:30.865543211 1974-06-27 05:58:30.865543211 1975-01-12 03:43:37.112456789
    +1976-03-03 04:54:33.000895162 1975-11-24 17:31:59.877438373 1976-06-10 17:17:06.124351951 1976-06-10 17:17:06.124351951 1975-11-24 17:31:59.877438373 1975-11-24 17:31:59.877438373 1976-06-10 17:17:06.124351951
    +1976-05-06 00:42:30.910786948 1976-01-27 12:19:57.787330159 1976-08-13 12:05:04.034243737 1976-08-13 12:05:04.034243737 1976-01-27 12:19:57.787330159 1976-01-27 12:19:57.787330159 1976-08-13 12:05:04.034243737
    +1978-08-05 14:41:05.501 1978-04-28 02:18:32.377543211 1978-11-13 01:03:38.624456789 1978-11-13 01:03:38.624456789 1978-04-28 02:18:32.377543211 1978-04-28 02:18:32.377543211 1978-11-13 01:03:38.624456789
    +1981-04-25 09:01:12.077192689 1981-01-15 21:38:38.9537359 1981-08-02 21:23:45.200649478 1981-08-02 21:23:45.200649478 1981-01-15 21:38:38.9537359 1981-01-15 21:38:38.9537359 1981-08-02 21:23:45.200649478
    +1981-11-15 23:03:10.999338387 1981-08-08 12:40:37.875881598 1982-02-23 10:25:44.122795176 1982-02-23 10:25:44.122795176 1981-08-08 12:40:37.875881598 1981-08-08 12:40:37.875881598 1982-02-23 10:25:44.122795176
    +1985-07-20 09:30:11 1985-04-11 21:07:37.876543211 1985-10-27 19:52:44.123456789 1985-10-27 19:52:44.123456789 1985-04-11 21:07:37.876543211 1985-04-11 21:07:37.876543211 1985-10-27 19:52:44.123456789
    +1985-11-18 16:37:54 1985-08-11 06:15:20.876543211 1986-02-26 04:00:27.123456789 1986-02-26 04:00:27.123456789 1985-08-11 06:15:20.876543211 1985-08-11 06:15:20.876543211 1986-02-26 04:00:27.123456789
    +1987-02-21 19:48:29 1986-11-14 08:25:55.876543211 1987-06-01 08:11:02.123456789 1987-06-01 08:11:02.123456789 1986-11-14 08:25:55.876543211 1986-11-14 08:25:55.876543211 1987-06-01 08:11:02.123456789
    +1987-05-28 13:52:07.900916635 1987-02-18 01:29:34.777459846 1987-09-05 01:14:41.024373424 1987-09-05 01:14:41.024373424 1987-02-18 01:29:34.777459846 1987-02-18 01:29:34.777459846 1987-09-05 01:14:41.024373424
    +1998-10-16 20:05:29.397591987 1998-07-09 08:42:56.274135198 1999-01-24 06:28:02.521048776 1999-01-24 06:28:02.521048776 1998-07-09 08:42:56.274135198 1998-07-09 08:42:56.274135198 1999-01-24 06:28:02.521048776
    +1999-10-03 16:59:10.396903939 1999-06-26 05:36:37.27344715 2000-01-11 03:21:43.520360728 2000-01-11 03:21:43.520360728 1999-06-26 05:36:37.27344715 1999-06-26 05:36:37.27344715 2000-01-11 03:21:43.520360728
    +2000-12-18 08:42:30.000595596 2000-09-09 22:19:56.877138807 2001-03-27 20:05:03.124052385 2001-03-27 20:05:03.124052385 2000-09-09 22:19:56.877138807 2000-09-09 22:19:56.877138807 2001-03-27 20:05:03.124052385
    +2002-05-10 05:29:48.990818073 2002-01-30 17:07:15.867361284 2002-08-17 16:52:22.114274862 2002-08-17 16:52:22.114274862 2002-01-30 17:07:15.867361284 2002-01-30 17:07:15.867361284 2002-08-17 16:52:22.114274862
    +2003-09-23 22:33:17.00003252 2003-06-16 11:10:43.876575731 2004-01-01 08:55:50.123489309 2004-01-01 08:55:50.123489309 2003-06-16 11:10:43.876575731 2003-06-16 11:10:43.876575731 2004-01-01 08:55:50.123489309
    +2004-03-07 20:14:13 2003-11-29 08:51:39.876543211 2004-06-15 08:36:46.123456789 2004-06-15 08:36:46.123456789 2003-11-29 08:51:39.876543211 2003-11-29 08:51:39.876543211 2004-06-15 08:36:46.123456789
    +2007-02-09 05:17:29.368756876 2006-11-01 17:54:56.245300087 2007-05-19 17:40:02.492213665 2007-05-19 17:40:02.492213665 2006-11-01 17:54:56.245300087 2006-11-01 17:54:56.245300087 2007-05-19 17:40:02.492213665
    +2009-01-21 10:49:07.108 2008-10-14 00:26:33.984543211 2009-04-30 23:11:40.231456789 2009-04-30 23:11:40.231456789 2008-10-14 00:26:33.984543211 2008-10-14 00:26:33.984543211 2009-04-30 23:11:40.231456789
    +2010-04-08 02:43:35.861742727 2009-12-29 14:21:02.738285938 2010-07-16 14:06:08.985199516 2010-07-16 14:06:08.985199516 2009-12-29 14:21:02.738285938 2009-12-29 14:21:02.738285938 2010-07-16 14:06:08.985199516
    +2013-04-07 02:44:43.00086821 2012-12-28 14:22:09.877411421 2013-07-15 14:07:16.124324999 2013-07-15 14:07:16.124324999 2012-12-28 14:22:09.877411421 2012-12-28 14:22:09.877411421 2013-07-15 14:07:16.124324999
    +2013-04-10 00:43:46.854731546 2012-12-31 12:21:13.731274757 2013-07-18 12:06:19.978188335 2013-07-18 12:06:19.978188335 2012-12-31 12:21:13.731274757 2012-12-31 12:21:13.731274757 2013-07-18 12:06:19.978188335
    +2021-09-24 03:18:32.413655165 2021-06-16 15:55:59.290198376 2022-01-01 13:41:05.537111954 2022-01-01 13:41:05.537111954 2021-06-16 15:55:59.290198376 2021-06-16 15:55:59.290198376 2022-01-01 13:41:05.537111954
    +2024-11-11 16:42:41.101 2024-08-04 06:20:07.977543211 2025-02-19 04:05:14.224456789 2025-02-19 04:05:14.224456789 2024-08-04 06:20:07.977543211 2024-08-04 06:20:07.977543211 2025-02-19 04:05:14.224456789
    +4143-07-08 10:53:27.252802259 4143-03-30 23:30:54.12934547 4143-10-15 22:16:00.376259048 4143-10-15 22:16:00.376259048 4143-03-30 23:30:54.12934547 4143-03-30 23:30:54.12934547 4143-10-15 22:16:00.376259048
    +4966-12-04 09:30:55.202 4966-08-26 23:08:22.078543211 4967-03-13 21:53:28.325456789 4967-03-13 21:53:28.325456789 4966-08-26 23:08:22.078543211 4966-08-26 23:08:22.078543211 4967-03-13 21:53:28.325456789
    +5339-02-01 14:10:01.085678691 5338-10-25 03:47:27.962221902 5339-05-12 02:32:34.20913548 5339-05-12 02:32:34.20913548 5338-10-25 03:47:27.962221902 5338-10-25 03:47:27.962221902 5339-05-12 02:32:34.20913548
    +5344-10-04 18:40:08.165 5344-06-27 07:17:35.041543211 5345-01-12 05:02:41.288456789 5345-01-12 05:02:41.288456789 5344-06-27 07:17:35.041543211 5344-06-27 07:17:35.041543211 5345-01-12 05:02:41.288456789
    +5397-07-13 07:12:32.000896438 5397-04-04 19:49:58.877439649 5397-10-20 18:35:05.124353227 5397-10-20 18:35:05.124353227 5397-04-04 19:49:58.877439649 5397-04-04 19:49:58.877439649 5397-10-20 18:35:05.124353227
    +5966-07-09 03:30:50.597 5966-03-31 16:08:17.473543211 5966-10-16 14:53:23.720456789 5966-10-16 14:53:23.720456789 5966-03-31 16:08:17.473543211 5966-03-31 16:08:17.473543211 5966-10-16 14:53:23.720456789
    +6229-06-28 02:54:28.970117179 6229-03-20 15:31:55.84666039 6229-10-05 14:17:02.093573968 6229-10-05 14:17:02.093573968 6229-03-20 15:31:55.84666039 6229-03-20 15:31:55.84666039 6229-10-05 14:17:02.093573968
    +6482-04-27 12:07:38.073915413 6482-01-17 23:45:04.950458624 6482-08-04 23:30:11.197372202 6482-08-04 23:30:11.197372202 6482-01-17 23:45:04.950458624 6482-01-17 23:45:04.950458624 6482-08-04 23:30:11.197372202
    +6631-11-13 16:31:29.702202248 6631-08-06 06:08:56.578745459 6632-02-21 03:54:02.825659037 6632-02-21 03:54:02.825659037 6631-08-06 06:08:56.578745459 6631-08-06 06:08:56.578745459 6632-02-21 03:54:02.825659037
    +6705-09-28 18:27:28.000845672 6705-06-21 07:04:54.877388883 6706-01-06 04:50:01.124302461 6706-01-06 04:50:01.124302461 6705-06-21 07:04:54.877388883 6705-06-21 07:04:54.877388883 6706-01-06 04:50:01.124302461
    +6731-02-12 08:12:48.287783702 6730-11-04 20:50:15.164326913 6731-05-22 20:35:21.411240491 6731-05-22 20:35:21.411240491 6730-11-04 20:50:15.164326913 6730-11-04 20:50:15.164326913 6731-05-22 20:35:21.411240491
    +7160-12-02 06:00:24.81200852 7160-08-24 19:37:51.688551731 7161-03-11 17:22:57.935465309 7161-03-11 17:22:57.935465309 7160-08-24 19:37:51.688551731 7160-08-24 19:37:51.688551731 7161-03-11 17:22:57.935465309
    +7409-09-07 23:33:32.459349602 7409-05-31 12:10:59.335892813 7409-12-16 09:56:05.582806391 7409-12-16 09:56:05.582806391 7409-05-31 12:10:59.335892813 7409-05-31 12:10:59.335892813 7409-12-16 09:56:05.582806391
    +7503-06-23 23:14:17.486 7503-03-16 11:51:44.362543211 7503-10-01 10:36:50.609456789 7503-10-01 10:36:50.609456789 7503-03-16 11:51:44.362543211 7503-03-16 11:51:44.362543211 7503-10-01 10:36:50.609456789
    +8422-07-22 03:21:45.745036084 8422-04-13 15:59:12.621579295 8422-10-29 14:44:18.868492873 8422-10-29 14:44:18.868492873 8422-04-13 15:59:12.621579295 8422-04-13 15:59:12.621579295 8422-10-29 14:44:18.868492873
    +8521-01-16 20:42:05.668832388 8520-10-09 10:19:32.545375599 8521-04-26 09:04:38.792289177 8521-04-26 09:04:38.792289177 8520-10-09 10:19:32.545375599 8520-10-09 10:19:32.545375599 8521-04-26 09:04:38.792289177
    +9075-06-13 16:20:09.218517797 9075-03-06 03:57:36.095061008 9075-09-21 03:42:42.341974586 9075-09-21 03:42:42.341974586 9075-03-06 03:57:36.095061008 9075-03-06 03:57:36.095061008 9075-09-21 03:42:42.341974586
    +9209-11-11 04:08:58.223768453 9209-08-03 17:46:25.100311664 9210-02-18 15:31:31.347225242 9210-02-18 15:31:31.347225242 9209-08-03 17:46:25.100311664 9209-08-03 17:46:25.100311664 9210-02-18 15:31:31.347225242
    +9403-01-09 18:12:33.547 9402-10-02 07:50:00.423543211 9403-04-19 06:35:06.670456789 9403-04-19 06:35:06.670456789 9402-10-02 07:50:00.423543211 9402-10-02 07:50:00.423543211 9403-04-19 06:35:06.670456789
    +PREHOOK: query: explain
    +select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +PREHOOK: type: QUERY
    +POSTHOOK: query: explain
    +select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +POSTHOOK: type: QUERY
    +Explain
    +STAGE DEPENDENCIES:
    + Stage-1 is a root stage
    + Stage-0 depends on stages: Stage-1
    +
    +STAGE PLANS:
    + Stage: Stage-1
    + Map Reduce
    + Map Operator Tree:
    + TableScan
    + alias: interval_arithmetic_1
    + Statistics: Num rows: 50 Data size: 4800 Basic stats: COMPLETE Column stats: COMPLETE
    + Select Operator
    + expressions: 109 20:30:40.246913578 (type: interval_day_time), 89 02:14:26.000000000 (type: interval_day_time)
    + outputColumnNames: _col0, _col1
    + Statistics: Num rows: 50 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + Limit
    + Number of rows: 2
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + File Output Operator
    + compressed: false
    + Statistics: Num rows: 2 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
    + table:
    + input format: org.apache.hadoop.mapred.SequenceFileInputFormat
    + output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
    + serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
    + Execution mode: vectorized
    +
    + Stage: Stage-0
    + Fetch Operator
    + limit: 2
    + Processor Tree:
    + ListSink
    +
    +PREHOOK: query: select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +PREHOOK: type: QUERY
    +PREHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +POSTHOOK: query: select
    + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second,
    + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second
    +from interval_arithmetic_1
    +limit 2
    +POSTHOOK: type: QUERY
    +POSTHOOK: Input: default@interval_arithmetic_1
    +#### A masked pattern was here ####
    +_c0 _c1
    +109 20:30:40.246913578 89 02:14:26.000000000
    +109 20:30:40.246913578 89 02:14:26.000000000
    +PREHOOK: query: drop table interval_arithmetic_1
    +PREHOOK: type: DROPTABLE
    +PREHOOK: Input: default@interval_arithmetic_1
    +PREHOOK: Output: default@interval_arithmetic_1
    +POSTHOOK: query: drop table interval_arithmetic_1
    +POSTHOOK: type: DROPTABLE
    +POSTHOOK: Input: default@interval_arithmetic_1
    +POSTHOOK: Output: default@interval_arithmetic_1

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/test/results/clientpositive/vectorized_casts.q.out
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/results/clientpositive/vectorized_casts.q.out b/ql/src/test/results/clientpositive/vectorized_casts.q.out
    index 6256400..804653c 100644
    --- a/ql/src/test/results/clientpositive/vectorized_casts.q.out
    +++ b/ql/src/test/results/clientpositive/vectorized_casts.q.out
    @@ -350,18 +350,18 @@ true NULL true true true NULL true false true true 11 NULL -64615982 1803053750
      true NULL true true true NULL true false true true 8 NULL 890988972 -1862301000 8 NULL 1 15 NULL NULL 8 8 8 8.0 NULL 8.90988972E8 -1.862301E9 8.0 NULL 1.0 15.892 NULL NULL 8.9098899E8 NULL 1969-12-31 16:00:00.008 NULL 1970-01-10 23:29:48.972 1969-12-10 02:41:39 1969-12-31 16:00:08 NULL 1969-12-31 16:00:00.001 1969-12-31 16:00:00 1969-12-31 16:00:15.892 NULL NULL 8 NULL 890988972 -1862301000 8.0 NULL TRUE 0 1969-12-31 16:00:15.892 XylAH4 XylAH4 XylAH4 8.0 1.781977944E9 0.9893582466233818 8.90988973E8
      true NULL true true true NULL true false true true 8 NULL 930867246 1205399250 8 NULL 1 15 NULL NULL 8 8 8 8.0 NULL 9.30867246E8 1.20539925E9 8.0 NULL 1.0 15.892 NULL NULL 9.3086726E8 NULL 1969-12-31 16:00:00.008 NULL 1970-01-11 10:34:27.246 1970-01-14 14:49:59.25 1969-12-31 16:00:08 NULL 1969-12-31 16:00:00.001 1969-12-31 16:00:00 1969-12-31 16:00:15.892 NULL NULL 8 NULL 930867246 1205399250 8.0 NULL TRUE 0 1969-12-31 16:00:15.892 c1V8o1A c1V8o1A c1V8o1A 8.0 1.861734492E9 0.9893582466233818 9.30867247E8
      true true NULL true true true NULL false true NULL -14 -7196 NULL -1552199500 -14 -7196 NULL 11 NULL NULL -14 -14 -14 -14.0 -7196.0 NULL -1.5521995E9 -14.0 -7196.0 NULL 11.065 NULL NULL NULL -7196.0 1969-12-31 15:59:59.986 1969-12-31 15:59:52.804 NULL 1969-12-13 16:50:00.5 1969-12-31 15:59:46 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:11.065 NULL NULL -14 -7196 NULL -1552199500 -14.0 -7196.0 NULL 0 1969-12-31 16:00:11.065 NULL NULL NULL -14.0 NULL -0.9906073556948704 NULL
    -true true NULL true true true NULL false true NULL -21 -7196 NULL 1542429000 -21 -7196 NULL -4 NULL NULL -21 -21 -21 -21.0 -7196.0 NULL 1.542429E9 -21.0 -7196.0 NULL -4.1 NULL NULL NULL -7196.0 1969-12-31 15:59:59.979 1969-12-31 15:59:52.804 NULL 1970-01-18 12:27:09 1969-12-31 15:59:39 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:55.9 NULL NULL -21 -7196 NULL 1542429000 -21.0 -7196.0 NULL 0 1969-12-31 15:59:55.9 NULL NULL NULL -21.0 NULL -0.8366556385360561 NULL
    -true true NULL true true true NULL false true NULL -24 -7196 NULL 829111000 -24 -7196 NULL -6 NULL NULL -24 -24 -24 -24.0 -7196.0 NULL 8.29111E8 -24.0 -7196.0 NULL -6.855 NULL NULL NULL -7196.0 1969-12-31 15:59:59.976 1969-12-31 15:59:52.804 NULL 1970-01-10 06:18:31 1969-12-31 15:59:36 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.145 NULL NULL -24 -7196 NULL 829111000 -24.0 -7196.0 NULL 0 1969-12-31 15:59:53.145 NULL NULL NULL -24.0 NULL 0.9055783620066238 NULL
    +true true NULL true true true NULL false true NULL -21 -7196 NULL 1542429000 -21 -7196 NULL -5 NULL NULL -21 -21 -21 -21.0 -7196.0 NULL 1.542429E9 -21.0 -7196.0 NULL -4.1 NULL NULL NULL -7196.0 1969-12-31 15:59:59.979 1969-12-31 15:59:52.804 NULL 1970-01-18 12:27:09 1969-12-31 15:59:39 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:55.9 NULL NULL -21 -7196 NULL 1542429000 -21.0 -7196.0 NULL 0 1969-12-31 15:59:55.9 NULL NULL NULL -21.0 NULL -0.8366556385360561 NULL
    +true true NULL true true true NULL false true NULL -24 -7196 NULL 829111000 -24 -7196 NULL -7 NULL NULL -24 -24 -24 -24.0 -7196.0 NULL 8.29111E8 -24.0 -7196.0 NULL -6.855 NULL NULL NULL -7196.0 1969-12-31 15:59:59.976 1969-12-31 15:59:52.804 NULL 1970-01-10 06:18:31 1969-12-31 15:59:36 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.145 NULL NULL -24 -7196 NULL 829111000 -24.0 -7196.0 NULL 0 1969-12-31 15:59:53.145 NULL NULL NULL -24.0 NULL 0.9055783620066238 NULL
      true true NULL true true true NULL false true NULL -30 -200 NULL 1429852250 -30 -200 NULL 12 NULL NULL -30 -30 -30 -30.0 -200.0 NULL 1.42985225E9 -30.0 -200.0 NULL 12.935 NULL NULL NULL -200.0 1969-12-31 15:59:59.97 1969-12-31 15:59:59.8 NULL 1970-01-17 05:10:52.25 1969-12-31 15:59:30 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 16:00:12.935 NULL NULL -30 -200 NULL 1429852250 -30.0 -200.0 NULL 0 1969-12-31 16:00:12.935 NULL NULL NULL -30.0 NULL 0.9880316240928618 NULL
    -true true NULL true true true NULL false true NULL -36 -200 NULL -2006216750 -36 -200 NULL -14 NULL NULL -36 -36 -36 -36.0 -200.0 NULL -2.00621675E9 -36.0 -200.0 NULL -14.252 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1969-12-08 10:43:03.25 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.748 NULL NULL -36 -200 NULL -2006216750 -36.0 -200.0 NULL 0 1969-12-31 15:59:45.748 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    -true true NULL true true true NULL false true NULL -36 -200 NULL 1599879000 -36 -200 NULL -6 NULL NULL -36 -36 -36 -36.0 -200.0 NULL 1.599879E9 -36.0 -200.0 NULL -6.183 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1970-01-19 04:24:39 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.817 NULL NULL -36 -200 NULL 1599879000 -36.0 -200.0 NULL 0 1969-12-31 15:59:53.817 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    -true true NULL true true true NULL false true NULL -38 15601 NULL -1858689000 -38 15601 NULL -1 NULL NULL -38 -38 -38 -38.0 15601.0 NULL -1.858689E9 -38.0 15601.0 NULL -1.386 NULL NULL NULL 15601.0 1969-12-31 15:59:59.962 1969-12-31 16:00:15.601 NULL 1969-12-10 03:41:51 1969-12-31 15:59:22 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:58.614 NULL NULL -38 15601 NULL -1858689000 -38.0 15601.0 NULL 0 1969-12-31 15:59:58.614 NULL NULL NULL -38.0 NULL -0.2963685787093853 NULL
    +true true NULL true true true NULL false true NULL -36 -200 NULL -2006216750 -36 -200 NULL -15 NULL NULL -36 -36 -36 -36.0 -200.0 NULL -2.00621675E9 -36.0 -200.0 NULL -14.252 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1969-12-08 10:43:03.25 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.748 NULL NULL -36 -200 NULL -2006216750 -36.0 -200.0 NULL 0 1969-12-31 15:59:45.748 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    +true true NULL true true true NULL false true NULL -36 -200 NULL 1599879000 -36 -200 NULL -7 NULL NULL -36 -36 -36 -36.0 -200.0 NULL 1.599879E9 -36.0 -200.0 NULL -6.183 NULL NULL NULL -200.0 1969-12-31 15:59:59.964 1969-12-31 15:59:59.8 NULL 1970-01-19 04:24:39 1969-12-31 15:59:24 1969-12-31 15:56:40 NULL 1969-12-31 16:00:00 1969-12-31 15:59:53.817 NULL NULL -36 -200 NULL 1599879000 -36.0 -200.0 NULL 0 1969-12-31 15:59:53.817 NULL NULL NULL -36.0 NULL 0.9917788534431158 NULL
    +true true NULL true true true NULL false true NULL -38 15601 NULL -1858689000 -38 15601 NULL -2 NULL NULL -38 -38 -38 -38.0 15601.0 NULL -1.858689E9 -38.0 15601.0 NULL -1.3860000000000001 NULL NULL NULL 15601.0 1969-12-31 15:59:59.962 1969-12-31 16:00:15.601 NULL 1969-12-10 03:41:51 1969-12-31 15:59:22 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:58.614 NULL NULL -38 15601 NULL -1858689000 -38.0 15601.0 NULL 0 1969-12-31 15:59:58.614 NULL NULL NULL -38.0 NULL -0.2963685787093853 NULL
      true true NULL true true true NULL false true NULL -5 15601 NULL 612416000 -5 15601 NULL 4 NULL NULL -5 -5 -5 -5.0 15601.0 NULL 6.12416E8 -5.0 15601.0 NULL 4.679 NULL NULL NULL 15601.0 1969-12-31 15:59:59.995 1969-12-31 16:00:15.601 NULL 1970-01-07 18:06:56 1969-12-31 15:59:55 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 16:00:04.679 NULL NULL -5 15601 NULL 612416000 -5.0 15601.0 NULL 0 1969-12-31 16:00:04.679 NULL NULL NULL -5.0 NULL 0.9589242746631385 NULL
    -true true NULL true true true NULL false true NULL -50 -7196 NULL -1031187250 -50 -7196 NULL -5 NULL NULL -50 -50 -50 -50.0 -7196.0 NULL -1.03118725E9 -50.0 -7196.0 NULL -5.267 NULL NULL NULL -7196.0 1969-12-31 15:59:59.95 1969-12-31 15:59:52.804 NULL 1969-12-19 17:33:32.75 1969-12-31 15:59:10 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:54.733 NULL NULL -50 -7196 NULL -1031187250 -50.0 -7196.0 NULL 0 1969-12-31 15:59:54.733 NULL NULL NULL -50.0 NULL 0.26237485370392877 NULL
    +true true NULL true true true NULL false true NULL -50 -7196 NULL -1031187250 -50 -7196 NULL -6 NULL NULL -50 -50 -50 -50.0 -7196.0 NULL -1.03118725E9 -50.0 -7196.0 NULL -5.267 NULL NULL NULL -7196.0 1969-12-31 15:59:59.95 1969-12-31 15:59:52.804 NULL 1969-12-19 17:33:32.75 1969-12-31 15:59:10 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:54.733 NULL NULL -50 -7196 NULL -1031187250 -50.0 -7196.0 NULL 0 1969-12-31 15:59:54.733 NULL NULL NULL -50.0 NULL 0.26237485370392877 NULL
      true true NULL true true true NULL false true NULL -59 -7196 NULL -1604890000 -59 -7196 NULL 13 NULL NULL -59 -59 -59 -59.0 -7196.0 NULL -1.60489E9 -59.0 -7196.0 NULL 13.15 NULL NULL NULL -7196.0 1969-12-31 15:59:59.941 1969-12-31 15:59:52.804 NULL 1969-12-13 02:11:50 1969-12-31 15:59:01 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:13.15 NULL NULL -59 -7196 NULL -1604890000 -59.0 -7196.0 NULL 0 1969-12-31 16:00:13.15 NULL NULL NULL -59.0 NULL -0.6367380071391379 NULL
    -true true NULL true true true NULL false true NULL -60 -7196 NULL 1516314750 -60 -7196 NULL -7 NULL NULL -60 -60 -60 -60.0 -7196.0 NULL 1.51631475E9 -60.0 -7196.0 NULL -7.592 NULL NULL NULL -7196.0 1969-12-31 15:59:59.94 1969-12-31 15:59:52.804 NULL 1970-01-18 05:11:54.75 1969-12-31 15:59:00 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:52.408 NULL NULL -60 -7196 NULL 1516314750 -60.0 -7196.0 NULL 0 1969-12-31 15:59:52.408 NULL NULL NULL -60.0 NULL 0.3048106211022167 NULL
    +true true NULL true true true NULL false true NULL -60 -7196 NULL 1516314750 -60 -7196 NULL -8 NULL NULL -60 -60 -60 -60.0 -7196.0 NULL 1.51631475E9 -60.0 -7196.0 NULL -7.592 NULL NULL NULL -7196.0 1969-12-31 15:59:59.94 1969-12-31 15:59:52.804 NULL 1970-01-18 05:11:54.75 1969-12-31 15:59:00 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 15:59:52.408 NULL NULL -60 -7196 NULL 1516314750 -60.0 -7196.0 NULL 0 1969-12-31 15:59:52.408 NULL NULL NULL -60.0 NULL 0.3048106211022167 NULL
      true true NULL true true true NULL false true NULL -8 -7196 NULL -1849991500 -8 -7196 NULL 3 NULL NULL -8 -8 -8 -8.0 -7196.0 NULL -1.8499915E9 -8.0 -7196.0 NULL 3.136 NULL NULL NULL -7196.0 1969-12-31 15:59:59.992 1969-12-31 15:59:52.804 NULL 1969-12-10 06:06:48.5 1969-12-31 15:59:52 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:03.136 NULL NULL -8 -7196 NULL -1849991500 -8.0 -7196.0 NULL 0 1969-12-31 16:00:03.136 NULL NULL NULL -8.0 NULL -0.9893582466233818 NULL
    -true true NULL true true true NULL false true NULL 20 15601 NULL -362433250 20 15601 NULL -14 NULL NULL 20 20 20 20.0 15601.0 NULL -3.6243325E8 20.0 15601.0 NULL -14.871 NULL NULL NULL 15601.0 1969-12-31 16:00:00.02 1969-12-31 16:00:15.601 NULL 1969-12-27 11:19:26.75 1969-12-31 16:00:20 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.129 NULL NULL 20 15601 NULL -362433250 20.0 15601.0 NULL 0 1969-12-31 15:59:45.129 NULL NULL NULL 20.0 NULL 0.9129452507276277 NULL
    -true true NULL true true true NULL false true NULL 48 15601 NULL -795361000 48 15601 NULL -9 NULL NULL 48 48 48 48.0 15601.0 NULL -7.95361E8 48.0 15601.0 NULL -9.765 NULL NULL NULL 15601.0 1969-12-31 16:00:00.048 1969-12-31 16:00:15.601 NULL 1969-12-22 11:03:59 1969-12-31 16:00:48 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:50.235 NULL NULL 48 15601 NULL -795361000 48.0 15601.0 NULL 0 1969-12-31 15:59:50.235 NULL NULL NULL 48.0 NULL -0.7682546613236668 NULL
    +true true NULL true true true NULL false true NULL 20 15601 NULL -362433250 20 15601 NULL -15 NULL NULL 20 20 20 20.0 15601.0 NULL -3.6243325E8 20.0 15601.0 NULL -14.871 NULL NULL NULL 15601.0 1969-12-31 16:00:00.02 1969-12-31 16:00:15.601 NULL 1969-12-27 11:19:26.75 1969-12-31 16:00:20 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:45.129 NULL NULL 20 15601 NULL -362433250 20.0 15601.0 NULL 0 1969-12-31 15:59:45.129 NULL NULL NULL 20.0 NULL 0.9129452507276277 NULL
    +true true NULL true true true NULL false true NULL 48 15601 NULL -795361000 48 15601 NULL -10 NULL NULL 48 48 48 48.0 15601.0 NULL -7.95361E8 48.0 15601.0 NULL -9.765 NULL NULL NULL 15601.0 1969-12-31 16:00:00.048 1969-12-31 16:00:15.601 NULL 1969-12-22 11:03:59 1969-12-31 16:00:48 1969-12-31 20:20:01 NULL 1969-12-31 16:00:00 1969-12-31 15:59:50.235 NULL NULL 48 15601 NULL -795361000 48.0 15601.0 NULL 0 1969-12-31 15:59:50.235 NULL NULL NULL 48.0 NULL -0.7682546613236668 NULL
      true true NULL true true true NULL false true NULL 5 -7196 NULL -1015607500 5 -7196 NULL 10 NULL NULL 5 5 5 5.0 -7196.0 NULL -1.0156075E9 5.0 -7196.0 NULL 10.973 NULL NULL NULL -7196.0 1969-12-31 16:00:00.005 1969-12-31 15:59:52.804 NULL 1969-12-19 21:53:12.5 1969-12-31 16:00:05 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:10.973 NULL NULL 5 -7196 NULL -1015607500 5.0 -7196.0 NULL 0 1969-12-31 16:00:10.973 NULL NULL NULL 5.0 NULL -0.9589242746631385 NULL
      true true NULL true true true NULL false true NULL 59 -7196 NULL -1137754500 59 -7196 NULL 10 NULL NULL 59 59 59 59.0 -7196.0 NULL -1.1377545E9 59.0 -7196.0 NULL 10.956 NULL NULL NULL -7196.0 1969-12-31 16:00:00.059 1969-12-31 15:59:52.804 NULL 1969-12-18 11:57:25.5 1969-12-31 16:00:59 1969-12-31 14:00:04 NULL 1969-12-31 16:00:00 1969-12-31 16:00:10.956 NULL NULL 59 -7196 NULL -1137754500 59.0 -7196.0 NULL 0 1969-12-31 16:00:10.956 NULL NULL NULL 59.0 NULL 0.6367380071391379 NULL
  • Jdere at Apr 4, 2016 at 8:36 pm
    HIVE-13371: Fix test failure of testHasNull in TestColumnStatistics running on Windows (PPengcheng Xiong, reviewed by Ashutosh Chauhan)


    Project: http://git-wip-us.apache.org/repos/asf/hive/repo
    Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/44ab4553
    Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/44ab4553
    Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/44ab4553

    Branch: refs/heads/llap
    Commit: 44ab45534277920bcf64dbd43409ab730fdb8d61
    Parents: 5201629
    Author: Pengcheng Xiong <pxiong@apache.org>
    Authored: Mon Mar 28 16:04:45 2016 -0700
    Committer: Pengcheng Xiong <pxiong@apache.org>
    Committed: Mon Mar 28 16:04:45 2016 -0700

    ----------------------------------------------------------------------
      .../org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java | 5 ++++-
      1 file changed, 4 insertions(+), 1 deletion(-)
    ----------------------------------------------------------------------


    http://git-wip-us.apache.org/repos/asf/hive/blob/44ab4553/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
    index 9433283..5f0146f 100644
    --- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
    +++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestColumnStatistics.java
    @@ -19,6 +19,7 @@
      package org.apache.hadoop.hive.ql.io.orc;

      import static junit.framework.Assert.assertEquals;
    +import static org.junit.Assume.assumeTrue;

      import java.io.File;
      import java.io.FileOutputStream;
    @@ -343,7 +344,9 @@ public class TestColumnStatistics {
          FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
          System.out.flush();
          System.setOut(origOut);
    -
    + // If called with an expression evaluating to false, the test will halt
    + // and be ignored.
    + assumeTrue(!System.getProperty("os.name").startsWith("Windows"));
          TestFileDump.checkOutput(outputFilename, workDir + File.separator + outputFilename);
        }
      }
  • Jdere at Apr 4, 2016 at 8:36 pm
    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
    index 2d7d0c2..bc09a3a 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/TimestampColumnInList.java
    @@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
      import java.sql.Timestamp;
      import java.util.HashSet;

    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
      import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor.Descriptor;
      import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
    @@ -36,11 +35,8 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
        private Timestamp[] inListValues;
        private int outputColumn;

    - private transient PisaTimestamp scratchTimestamp;
    -
    -
        // The set object containing the IN list.
    - private transient HashSet<PisaTimestamp> inSet;
    + private transient HashSet<Timestamp> inSet;

        public TimestampColumnInList() {
          super();
    @@ -64,11 +60,10 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
          }

          if (inSet == null) {
    - inSet = new HashSet<PisaTimestamp>(inListValues.length);
    + inSet = new HashSet<Timestamp>(inListValues.length);
            for (Timestamp val : inListValues) {
    - inSet.add(new PisaTimestamp(val));
    + inSet.add(val);
            }
    - scratchTimestamp = new PisaTimestamp();
          }

          TimestampColumnVector inputColVector = (TimestampColumnVector) batch.cols[inputCol];
    @@ -91,19 +86,16 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam

              // All must be selected otherwise size would be zero
              // Repeating property will not change.
    - inputColVector.pisaTimestampUpdate(scratchTimestamp, 0);
    - outputVector[0] = inSet.contains(scratchTimestamp) ? 1 : 0;
    + outputVector[0] = inSet.contains(inputColVector.asScratchTimestamp(0)) ? 1 : 0;
              outputColVector.isRepeating = true;
            } else if (batch.selectedInUse) {
              for(int j = 0; j != n; j++) {
                int i = sel[j];
    - inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
    - outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
    + outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
              }
            } else {
              for(int i = 0; i != n; i++) {
    - inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
    - outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
    + outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
              }
            }
          } else {
    @@ -112,8 +104,7 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
              //All must be selected otherwise size would be zero
              //Repeating property will not change.
              if (!nullPos[0]) {
    - inputColVector.pisaTimestampUpdate(scratchTimestamp, 0);
    - outputVector[0] = inSet.contains(scratchTimestamp) ? 1 : 0;
    + outputVector[0] = inSet.contains(inputColVector.asScratchTimestamp(0)) ? 1 : 0;
                outNulls[0] = false;
              } else {
                outNulls[0] = true;
    @@ -124,16 +115,14 @@ public class TimestampColumnInList extends VectorExpression implements ITimestam
                int i = sel[j];
                outNulls[i] = nullPos[i];
                if (!nullPos[i]) {
    - inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
    - outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
    + outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
                }
              }
            } else {
              System.arraycopy(nullPos, 0, outNulls, 0, n);
              for(int i = 0; i != n; i++) {
                if (!nullPos[i]) {
    - inputColVector.pisaTimestampUpdate(scratchTimestamp, i);
    - outputVector[i] = inSet.contains(scratchTimestamp) ? 1 : 0;
    + outputVector[i] = inSet.contains(inputColVector.asScratchTimestamp(i)) ? 1 : 0;
                }
              }
            }

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
    index 326bfb9..85dacd7 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriter.java
    @@ -21,10 +21,11 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
      import java.sql.Timestamp;

      import org.apache.hadoop.hive.common.type.HiveDecimal;
    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
    +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
      import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
      import org.apache.hadoop.hive.ql.metadata.HiveException;
      import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
    +import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
      import org.apache.hadoop.hive.serde2.io.TimestampWritable;
      import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;

    @@ -42,7 +43,8 @@ public interface VectorExpressionWriter {
        Object writeValue(HiveDecimal value) throws HiveException;
        Object writeValue(TimestampWritable value) throws HiveException;
        Object writeValue(Timestamp value) throws HiveException;
    - Object writeValue(PisaTimestamp value) throws HiveException;
    + Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException;
    + Object writeValue(HiveIntervalDayTime value) throws HiveException;
        Object setValue(Object row, ColumnVector column, int columnRow) throws HiveException;
        Object initValue(Object ost) throws HiveException;
      }

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
    index 9a1d7f3..c20bc68 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorExpressionWriterFactory.java
    @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.common.type.HiveDecimal;
      import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
      import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
      import org.apache.hadoop.hive.common.type.HiveVarchar;
    -import org.apache.hadoop.hive.common.type.PisaTimestamp;
      import org.apache.hadoop.hive.ql.exec.vector.*;
      import org.apache.hadoop.hive.ql.metadata.HiveException;
      import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
    @@ -188,17 +187,39 @@ public final class VectorExpressionWriterFactory {
          }

          /**
    - * The base implementation must be overridden by the PisaTimestamp specialization
    + * The base implementation must be overridden by the Timestamp specialization
    + */
    + public Object setValue(Object field, Timestamp value) throws HiveException {
    + throw new HiveException("Internal error: should not reach here");
    + }
    +
    + /**
    + * The base implementation must be overridden by the HiveIntervalDayTime specialization
           */
          @Override
    - public Object writeValue(PisaTimestamp value) throws HiveException {
    + public Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException {
            throw new HiveException("Internal error: should not reach here");
          }

          /**
    - * The base implementation must be overridden by the Timestamp specialization
    + * The base implementation must be overridden by the HiveIntervalDayTime specialization
           */
    - public Object setValue(Object field, Timestamp value) throws HiveException {
    + @Override
    + public Object writeValue(HiveIntervalDayTime value) throws HiveException {
    + throw new HiveException("Internal error: should not reach here");
    + }
    +
    + /**
    + * The base implementation must be overridden by the HiveIntervalDayTime specialization
    + */
    + public Object setValue(Object field, HiveIntervalDayTimeWritable value) throws HiveException {
    + throw new HiveException("Internal error: should not reach here");
    + }
    +
    + /**
    + * The base implementation must be overridden by the HiveIntervalDayTime specialization
    + */
    + public Object setValue(Object field, HiveIntervalDayTime value) throws HiveException {
            throw new HiveException("Internal error: should not reach here");
          }
        }
    @@ -465,6 +486,66 @@ public final class VectorExpressionWriterFactory {
          }
        }

    + /**
    + * Specialized writer for IntervalDayTimeColumnVector. Will throw cast exception
    + * if the wrong vector column is used.
    + */
    + private static abstract class VectorExpressionWriterIntervalDayTime extends VectorExpressionWriterBase {
    + @Override
    + public Object writeValue(ColumnVector column, int row) throws HiveException {
    + IntervalDayTimeColumnVector dcv = (IntervalDayTimeColumnVector) column;
    + HiveIntervalDayTimeWritable intervalDayTimeWritable = (HiveIntervalDayTimeWritable) dcv.getScratchWritable();
    + if (intervalDayTimeWritable == null) {
    + intervalDayTimeWritable = new HiveIntervalDayTimeWritable();
    + dcv.setScratchWritable(intervalDayTimeWritable);
    + }
    + if (dcv.noNulls && !dcv.isRepeating) {
    + return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
    + } else if (dcv.noNulls && dcv.isRepeating) {
    + return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
    + } else if (!dcv.noNulls && !dcv.isRepeating && !dcv.isNull[row]) {
    + return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
    + } else if (!dcv.noNulls && dcv.isRepeating && !dcv.isNull[0]) {
    + return writeValue(TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
    + } else if (!dcv.noNulls && dcv.isRepeating && dcv.isNull[0]) {
    + return null;
    + } else if (!dcv.noNulls && !dcv.isRepeating && dcv.isNull[row]) {
    + return null;
    + }
    + throw new HiveException(
    + String.format(
    + "Incorrect null/repeating: row:%d noNulls:%b isRepeating:%b isNull[row]:%b isNull[0]:%b",
    + row, dcv.noNulls, dcv.isRepeating, dcv.isNull[row], dcv.isNull[0]));
    + }
    +
    + @Override
    + public Object setValue(Object field, ColumnVector column, int row) throws HiveException {
    + IntervalDayTimeColumnVector dcv = (IntervalDayTimeColumnVector) column;
    + HiveIntervalDayTimeWritable intervalDayTimeWritable = (HiveIntervalDayTimeWritable) dcv.getScratchWritable();
    + if (intervalDayTimeWritable == null) {
    + intervalDayTimeWritable = new HiveIntervalDayTimeWritable();
    + dcv.setScratchWritable(intervalDayTimeWritable);
    + }
    + if (dcv.noNulls && !dcv.isRepeating) {
    + return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
    + } else if (dcv.noNulls && dcv.isRepeating) {
    + return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
    + } else if (!dcv.noNulls && !dcv.isRepeating && !dcv.isNull[row]) {
    + return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, row, intervalDayTimeWritable));
    + } else if (!dcv.noNulls && !dcv.isRepeating && dcv.isNull[row]) {
    + return null;
    + } else if (!dcv.noNulls && dcv.isRepeating && !dcv.isNull[0]) {
    + return setValue(field, TimestampUtils.intervalDayTimeColumnVectorWritable(dcv, 0, intervalDayTimeWritable));
    + } else if (!dcv.noNulls && dcv.isRepeating && dcv.isNull[0]) {
    + return null;
    + }
    + throw new HiveException(
    + String.format(
    + "Incorrect null/repeating: row:%d noNulls:%b isRepeating:%b isNull[row]:%b isNull[0]:%b",
    + row, dcv.noNulls, dcv.isRepeating, dcv.isNull[row], dcv.isNull[0]));
    + }
    + }
    +
          /**
           * Compiles the appropriate vector expression writer based on an expression info (ExprNodeDesc)
           */
    @@ -697,8 +778,13 @@ public final class VectorExpressionWriterFactory {
            }

            @Override
    - public Object writeValue(PisaTimestamp value) throws HiveException {
    - return ((SettableTimestampObjectInspector) this.objectInspector).set(obj, value.asScratchTimestamp());
    + public Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException {
    + return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, value);
    + }
    +
    + @Override
    + public Object writeValue(HiveIntervalDayTime value) throws HiveException {
    + return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, value);
            }

            @Override
    @@ -766,53 +852,45 @@ public final class VectorExpressionWriterFactory {
        private static VectorExpressionWriter genVectorExpressionWritableIntervalDayTime(
            SettableHiveIntervalDayTimeObjectInspector fieldObjInspector) throws HiveException {

    - return new VectorExpressionWriterTimestamp() {
    + return new VectorExpressionWriterIntervalDayTime() {
            private Object obj;
            private HiveIntervalDayTime interval;
    - private PisaTimestamp pisaTimestamp;

            public VectorExpressionWriter init(SettableHiveIntervalDayTimeObjectInspector objInspector)
                throws HiveException {
              super.init(objInspector);
              interval = new HiveIntervalDayTime();
              obj = initValue(null);
    - pisaTimestamp = new PisaTimestamp();
              return this;
            }

            @Override
    - public Object writeValue(TimestampWritable value) throws HiveException {
    - interval.set(pisaTimestamp.updateFromTimestamp(value.getTimestamp()));
    + public Object writeValue(HiveIntervalDayTimeWritable value) throws HiveException {
    + interval.set(value.getHiveIntervalDayTime());
              return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, interval);
            }

            @Override
    - public Object writeValue(Timestamp value) throws HiveException {
    - interval.set(pisaTimestamp.updateFromTimestamp(value));
    - return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, interval);
    - }
    -
    - @Override
    - public Object writeValue(PisaTimestamp value) throws HiveException {
    + public Object writeValue(HiveIntervalDayTime value) throws HiveException {
              interval.set(value);
              return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(obj, interval);
            }

            @Override
    - public Object setValue(Object field, TimestampWritable value) {
    + public Object setValue(Object field, HiveIntervalDayTimeWritable value) {
              if (null == field) {
                field = initValue(null);
              }
    - interval.set(pisaTimestamp.updateFromTimestamp(value.getTimestamp()));
    + interval.set(value.getHiveIntervalDayTime());
              return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(field, interval);
            }

            @Override
    - public Object setValue(Object field, Timestamp value) {
    + public Object setValue(Object field, HiveIntervalDayTime value) {
              if (null == field) {
                field = initValue(null);
              }
    - interval.set(pisaTimestamp.updateFromTimestamp(value));
    + interval.set(value);
              return ((SettableHiveIntervalDayTimeObjectInspector) this.objectInspector).set(field, interval);
            }


    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
    index 9f5c793..05dd93e 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColCol.java
    @@ -166,7 +166,7 @@ public class VectorUDFDateAddColCol extends VectorExpression {

        protected byte[] evaluateTimestamp(ColumnVector columnVector, int index, long numDays) {
          TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
    - calendar.setTimeInMillis(tcv.getTimestampMilliseconds(index));
    + calendar.setTimeInMillis(tcv.getTime(index));
          if (isPositive) {
            calendar.add(Calendar.DATE, (int) numDays);
          } else {

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
    index 6390ecd..59ca61e 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateAddColScalar.java
    @@ -210,7 +210,7 @@ public class VectorUDFDateAddColScalar extends VectorExpression {

        protected byte[] evaluateTimestamp(ColumnVector columnVector, int index) {
          TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
    - calendar.setTimeInMillis(tcv.getTimestampMilliseconds(index));
    + calendar.setTimeInMillis(tcv.getTime(index));
          if (isPositive) {
            calendar.add(Calendar.DATE, numDays);
          } else {

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
    index b22c31f..4edf558 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColCol.java
    @@ -275,7 +275,7 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
            output.isRepeating = true;

            if (!input.isNull[0]) {
    - date.setTime(input.getTimestampMilliseconds(0));
    + date.setTime(input.getTime(0));
              output.vector[0] = DateWritable.dateToDays(date);
            }
            return;
    @@ -288,12 +288,12 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
            if (selectedInUse) {
              for (int j = 0; j < size; j++) {
                int i = sel[j];
    - date.setTime(input.getTimestampMilliseconds(i));
    + date.setTime(input.getTime(i));
                output.vector[i] = DateWritable.dateToDays(date);
              }
            } else {
              for (int i = 0; i < size; i++) {
    - date.setTime(input.getTimestampMilliseconds(i));
    + date.setTime(input.getTime(i));
                output.vector[i] = DateWritable.dateToDays(date);
              }
            }
    @@ -312,14 +312,14 @@ public class VectorUDFDateDiffColCol extends VectorExpression {
              for (int j = 0; j < size; j++) {
                int i = sel[j];
                if (!input.isNull[i]) {
    - date.setTime(input.getTimestampMilliseconds(i));
    + date.setTime(input.getTime(i));
                  output.vector[i] = DateWritable.dateToDays(date);
                }
              }
            } else {
              for (int i = 0; i < size; i++) {
                if (!input.isNull[i]) {
    - date.setTime(input.getTimestampMilliseconds(i));
    + date.setTime(input.getTime(i));
                  output.vector[i] = DateWritable.dateToDays(date);
                }
              }

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
    index ab71b47..71b3887 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffColScalar.java
    @@ -238,7 +238,7 @@ public class VectorUDFDateDiffColScalar extends VectorExpression {

        protected int evaluateTimestamp(ColumnVector columnVector, int index) {
          TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
    - date.setTime(tcv.getTimestampMilliseconds(index));
    + date.setTime(tcv.getTime(index));
          return DateWritable.dateToDays(date) - baseDate;
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
    index dea5444..c733bc9 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateDiffScalarCol.java
    @@ -237,7 +237,7 @@ public class VectorUDFDateDiffScalarCol extends VectorExpression {

        protected int evaluateTimestamp(ColumnVector columnVector, int index) {
          TimestampColumnVector tcv = (TimestampColumnVector) columnVector;
    - date.setTime(tcv.getTimestampMilliseconds(index));
    + date.setTime(tcv.getTime(index));
          return baseDate - DateWritable.dateToDays(date);
        }


    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
    index c29e22e..cde0be4 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFDateTimestamp.java
    @@ -45,7 +45,7 @@ public class VectorUDFDateTimestamp extends TimestampToStringUnaryUDF {
        protected void func(BytesColumnVector outV, TimestampColumnVector inV, int i) {
          switch (inputTypes[0]) {
            case TIMESTAMP:
    - date.setTime(inV.getTimestampMilliseconds(i));
    + date.setTime(inV.getTime(i));
              break;

            default:

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
    index b7c4ff4..3c693af 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampDate.java
    @@ -28,15 +28,18 @@ public final class VectorUDFUnixTimeStampDate extends VectorUDFTimestampFieldDat

        private static final long serialVersionUID = 1L;

    + private DateWritable dateWritable;
    +
        @Override
        protected long getDateField(long days) {
    - long ms = DateWritable.daysToMillis((int) days);
    - return ms / 1000;
    + dateWritable.set((int) days);
    + return dateWritable.getTimeInSeconds();
        }

        public VectorUDFUnixTimeStampDate(int colNum, int outputColumn) {
          /* not a real field */
          super(-1, colNum, outputColumn);
    + dateWritable = new DateWritable();
        }

        public VectorUDFUnixTimeStampDate() {

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
    index e4a31ca..2bd7756 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/VectorUDFUnixTimeStampTimestamp.java
    @@ -18,10 +18,7 @@

      package org.apache.hadoop.hive.ql.exec.vector.expressions;

    -import java.sql.Timestamp;
    -
      import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
    -import org.apache.hadoop.hive.serde2.io.DateWritable;

      /**
       * Return Unix Timestamp.
    @@ -33,7 +30,7 @@ public final class VectorUDFUnixTimeStampTimestamp extends VectorUDFTimestampFie

        @Override
        protected long getTimestampField(TimestampColumnVector timestampColVector, int elementNum) {
    - return timestampColVector.getTimestampSeconds(elementNum);
    + return timestampColVector.asScratchTimestamp(elementNum).getTime() / 1000;
        }

        public VectorUDFUnixTimeStampTimestamp(int colNum, int outputColumn) {

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
    index 5c8db41..d0a1d0d 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgTimestamp.java
    @@ -27,8 +27,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggreg
      import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
      import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
    -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
    -import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
      import org.apache.hadoop.hive.ql.metadata.HiveException;
      import org.apache.hadoop.hive.ql.plan.AggregationDesc;
      import org.apache.hadoop.hive.ql.util.JavaDataModel;
    @@ -146,7 +144,8 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
              if (inputColVector.isRepeating) {
                iterateNoNullsRepeatingWithAggregationSelection(
                  aggregationBufferSets, bufferIndex,
    - inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
    + inputColVector.getDouble(0),
    + batchSize);
              } else {
                if (batch.selectedInUse) {
                  iterateNoNullsSelectionWithAggregationSelection(
    @@ -163,11 +162,11 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
                if (batch.selectedInUse) {
                  iterateHasNullsRepeatingSelectionWithAggregationSelection(
                    aggregationBufferSets, bufferIndex,
    - inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize, batch.selected, inputColVector.isNull);
    + inputColVector.getDouble(0), batchSize, batch.selected, inputColVector.isNull);
                } else {
                  iterateHasNullsRepeatingWithAggregationSelection(
                    aggregationBufferSets, bufferIndex,
    - inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize, inputColVector.isNull);
    + inputColVector.getDouble(0), batchSize, inputColVector.isNull);
                }
              } else {
                if (batch.selectedInUse) {
    @@ -210,7 +209,8 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
                aggregationBufferSets,
                bufferIndex,
                i);
    - myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(selection[i]));
    + myagg.sumValue(
    + inputColVector.getDouble(selection[i]));
            }
          }

    @@ -224,7 +224,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
                aggregationBufferSets,
                bufferIndex,
                i);
    - myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(i));
    + myagg.sumValue(inputColVector.getDouble(i));
            }
          }

    @@ -281,7 +281,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
                  aggregationBufferSets,
                  bufferIndex,
                  j);
    - myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(i));
    + myagg.sumValue(inputColVector.getDouble(i));
              }
            }
         }
    @@ -296,10 +296,10 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
            for (int i=0; i < batchSize; ++i) {
              if (!isNull[i]) {
                Aggregation myagg = getCurrentAggregationBuffer(
    - aggregationBufferSets,
    + aggregationBufferSets,
                  bufferIndex,
                  i);
    - myagg.sumValue(inputColVector.getTimestampSecondsWithFractionalNanos(i));
    + myagg.sumValue(inputColVector.getDouble(i));
              }
            }
         }
    @@ -328,7 +328,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
                    myagg.sum = 0;
                    myagg.count = 0;
                  }
    - myagg.sum += inputColVector.getTimestampSecondsWithFractionalNanos(0)*batchSize;
    + myagg.sum += inputColVector.getDouble(0)*batchSize;
                  myagg.count += batchSize;
                }
                return;
    @@ -358,7 +358,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
            for (int j=0; j< batchSize; ++j) {
              int i = selected[j];
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.isNull = false;
                  myagg.sum = 0;
    @@ -381,24 +381,24 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
              myagg.sum = 0;
              myagg.count = 0;
            }
    -
    +
            for (int i=0; i< batchSize; ++i) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
    + double value = inputColVector.getDouble(selected[i]);
              myagg.sum += value;
              myagg.count += 1;
            }
          }

          private void iterateNoSelectionHasNulls(
    - Aggregation myagg,
    - TimestampColumnVector inputColVector,
    + Aggregation myagg,
    + TimestampColumnVector inputColVector,
              int batchSize,
              boolean[] isNull) {
    -
    +
            for(int i=0;i<batchSize;++i) {
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    - if (myagg.isNull) {
    + double value = inputColVector.getDouble(i);
    + if (myagg.isNull) {
                  myagg.isNull = false;
                  myagg.sum = 0;
                  myagg.count = 0;
    @@ -420,7 +420,7 @@ public class VectorUDAFAvgTimestamp extends VectorAggregateExpression {
            }

            for (int i=0;i<batchSize;++i) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
              myagg.sum += value;
              myagg.count += 1;
            }

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
    index 17906ec..fa25e6a 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdPopTimestamp.java
    @@ -152,7 +152,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
            if (inputColVector.isRepeating) {
              if (inputColVector.noNulls || !inputColVector.isNull[0]) {
                iterateRepeatingNoNullsWithAggregationSelection(
    - aggregationBufferSets, aggregateIndex, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
    + aggregationBufferSets, aggregateIndex, inputColVector.getDouble(0), batchSize);
              }
            }
            else if (!batch.selectedInUse && inputColVector.noNulls) {
    @@ -213,7 +213,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
                j);
              int i = selected[j];
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -239,7 +239,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
                aggregationBufferSets,
                aggregateIndex,
                i);
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
    + double value = inputColVector.getDouble(selected[i]);
              if (myagg.isNull) {
                myagg.init ();
              }
    @@ -265,7 +265,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
                  aggregationBufferSets,
                  aggregateIndex,
                i);
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -293,7 +293,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
              if (myagg.isNull) {
                myagg.init ();
              }
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
              myagg.sum += value;
              myagg.count += 1;
              if(myagg.count > 1) {
    @@ -322,7 +322,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {

            if (inputColVector.isRepeating) {
              if (inputColVector.noNulls) {
    - iterateRepeatingNoNulls(myagg, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
    + iterateRepeatingNoNulls(myagg, inputColVector.getDouble(0), batchSize);
              }
            }
            else if (!batch.selectedInUse && inputColVector.noNulls) {
    @@ -377,7 +377,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
            for (int j=0; j< batchSize; ++j) {
              int i = selected[j];
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -401,7 +401,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
              myagg.init ();
            }

    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[0]);
    + double value = inputColVector.getDouble(selected[0]);
            myagg.sum += value;
            myagg.count += 1;
            if(myagg.count > 1) {
    @@ -412,7 +412,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
            // i=0 was pulled out to remove the count > 1 check in the loop
            //
            for (int i=1; i< batchSize; ++i) {
    - value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
    + value = inputColVector.getDouble(selected[i]);
              myagg.sum += value;
              myagg.count += 1;
              double t = myagg.count*value - myagg.sum;
    @@ -428,7 +428,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {

            for(int i=0;i<batchSize;++i) {
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -451,7 +451,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {
              myagg.init ();
            }

    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(0);
    + double value = inputColVector.getDouble(0);
            myagg.sum += value;
            myagg.count += 1;

    @@ -462,7 +462,7 @@ public class VectorUDAFStdPopTimestamp extends VectorAggregateExpression {

            // i=0 was pulled out to remove count > 1 check
            for (int i=1; i<batchSize; ++i) {
    - value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + value = inputColVector.getDouble(i);
              myagg.sum += value;
              myagg.count += 1;
              double t = myagg.count*value - myagg.sum;

    http://git-wip-us.apache.org/repos/asf/hive/blob/52016296/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
    ----------------------------------------------------------------------
    diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
    index 2e41e47..b3e1fae 100644
    --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
    +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFStdSampTimestamp.java
    @@ -26,7 +26,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
      import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorAggregateExpression;
      import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
      import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
    -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
      import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
      import org.apache.hadoop.hive.ql.metadata.HiveException;
      import org.apache.hadoop.hive.ql.plan.AggregationDesc;
    @@ -38,7 +37,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
      import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;

      /**
    -* VectorUDAFStdSampDouble. Vectorized implementation for VARIANCE aggregates.
    +* VectorUDAFStdSampTimestamp. Vectorized implementation for VARIANCE aggregates.
      */
      @Description(name = "stddev_samp",
          value = "_FUNC_(x) - Returns the sample standard deviation of a set of numbers (vectorized, double)")
    @@ -153,7 +152,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
            if (inputColVector.isRepeating) {
              if (inputColVector.noNulls || !inputColVector.isNull[0]) {
                iterateRepeatingNoNullsWithAggregationSelection(
    - aggregationBufferSets, aggregateIndex, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
    + aggregationBufferSets, aggregateIndex, inputColVector.getDouble(0), batchSize);
              }
            }
            else if (!batch.selectedInUse && inputColVector.noNulls) {
    @@ -214,7 +213,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
                j);
              int i = selected[j];
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -240,7 +239,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
                aggregationBufferSets,
                aggregateIndex,
                i);
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
    + double value = inputColVector.getDouble(selected[i]);
              if (myagg.isNull) {
                myagg.init ();
              }
    @@ -266,7 +265,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
                  aggregationBufferSets,
                  aggregateIndex,
                i);
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -294,7 +293,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
              if (myagg.isNull) {
                myagg.init ();
              }
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
              myagg.sum += value;
              myagg.count += 1;
              if(myagg.count > 1) {
    @@ -323,7 +322,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {

            if (inputColVector.isRepeating) {
              if (inputColVector.noNulls) {
    - iterateRepeatingNoNulls(myagg, inputColVector.getTimestampSecondsWithFractionalNanos(0), batchSize);
    + iterateRepeatingNoNulls(myagg, inputColVector.getDouble(0), batchSize);
              }
            }
            else if (!batch.selectedInUse && inputColVector.noNulls) {
    @@ -378,7 +377,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
            for (int j=0; j< batchSize; ++j) {
              int i = selected[j];
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -402,7 +401,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
              myagg.init ();
            }

    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[0]);
    + double value = inputColVector.getDouble(selected[0]);
            myagg.sum += value;
            myagg.count += 1;
            if(myagg.count > 1) {
    @@ -413,7 +412,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
            // i=0 was pulled out to remove the count > 1 check in the loop
            //
            for (int i=1; i< batchSize; ++i) {
    - value = inputColVector.getTimestampSecondsWithFractionalNanos(selected[i]);
    + value = inputColVector.getDouble(selected[i]);
              myagg.sum += value;
              myagg.count += 1;
              double t = myagg.count*value - myagg.sum;
    @@ -429,7 +428,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {

            for(int i=0;i<batchSize;++i) {
              if (!isNull[i]) {
    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(i);
    + double value = inputColVector.getDouble(i);
                if (myagg.isNull) {
                  myagg.init ();
                }
    @@ -452,7 +451,7 @@ public class VectorUDAFStdSampTimestamp extends VectorAggregateExpression {
              myagg.init ();
            }

    - double value = inputColVector.getTimestampSecondsWithFractionalNanos(0);
    + double value = inputColVector.getDouble(0);
            myagg.sum += value;
      &nbs