You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by to...@apache.org on 2008/09/19 12:19:23 UTC

svn commit: r697035 - in /hadoop/core/trunk: ./ src/contrib/hive/ src/contrib/hive/ql/ src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/ src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/ src/contrib/hive/ql/src/java/org/apache/ha...

Author: tomwhite
Date: Fri Sep 19 03:19:22 2008
New Revision: 697035

URL: http://svn.apache.org/viewvc?rev=697035&view=rev
Log:
HADOOP-4070. Provide a mechanism in Hive for registering UDFs from the query language.

Added:
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/FunctionWork.java
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java
    hadoop/core/trunk/src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/udf/
    hadoop/core/trunk/src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/udf/UDFTestLength.java
    hadoop/core/trunk/src/contrib/hive/ql/src/test/queries/clientpositive/udf2.q
    hadoop/core/trunk/src/contrib/hive/ql/src/test/results/clientpositive/udf2.q.out
Modified:
    hadoop/core/trunk/CHANGES.txt
    hadoop/core/trunk/src/contrib/hive/build-common.xml
    hadoop/core/trunk/src/contrib/hive/ql/build.xml
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
    hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java

Modified: hadoop/core/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/CHANGES.txt?rev=697035&r1=697034&r2=697035&view=diff
==============================================================================
--- hadoop/core/trunk/CHANGES.txt (original)
+++ hadoop/core/trunk/CHANGES.txt Fri Sep 19 03:19:22 2008
@@ -185,6 +185,9 @@
       add mapred.job.reuse.jvm.num.tasks
     (Devaraj Das via acmurthy) 
 
+    HADOOP-4070. Provide a mechanism in Hive for registering UDFs from the
+    query language. (tomwhite)
+
   IMPROVEMENTS
 
     HADOOP-4106. libhdfs: add time, permission and user attribute support (part 2).

Modified: hadoop/core/trunk/src/contrib/hive/build-common.xml
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/build-common.xml?rev=697035&r1=697034&r2=697035&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/build-common.xml (original)
+++ hadoop/core/trunk/src/contrib/hive/build-common.xml Fri Sep 19 03:19:22 2008
@@ -169,6 +169,8 @@
       <classpath refid="test.classpath"/>
     </javac>
   </target>
+	
+  <target name="test-jar" depends="compile-test"/>
 
   <target name="test-conditions">
     <condition property="qfile" value="">
@@ -200,7 +202,8 @@
   <target name="gen-test"/>
 
   <!-- target to run the tests -->
-  <target name="test" depends="test-conditions,gen-test,compile-test,test-init">
+  <target name="test"
+  	depends="test-conditions,gen-test,compile-test,test-jar,test-init">
     <delete dir="${test.log.dir}"/>
     <mkdir dir="${test.log.dir}"/>
     <junit showoutput="${test.output}" printsummary="yes" haltonfailure="no"
@@ -212,6 +215,7 @@
       <sysproperty key="test.output.overwrite" value="${overwrite}"/>
       <sysproperty key="log4j.configuration" value="file://${test.data.dir}/conf/hive-log4j.properties"/>
       <sysproperty key="derby.stream.error.file" value="${test.build.dir}/derby.log"/>
+      <sysproperty key="hive.aux.jars.path" value="file://${test.build.dir}/test-udfs.jar"/>
       <classpath refid="${test.classpath.id}"/>
       <formatter type="${test.junit.output.format}" />
       <batchtest todir="${test.build.dir}" unless="testcase">

Modified: hadoop/core/trunk/src/contrib/hive/ql/build.xml
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/build.xml?rev=697035&r1=697034&r2=697035&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/build.xml (original)
+++ hadoop/core/trunk/src/contrib/hive/ql/build.xml Fri Sep 19 03:19:22 2008
@@ -98,6 +98,13 @@
       <classpath refid="classpath"/>
     </javac>
   </target>
+	
+  <target name="test-jar" depends="compile-test">
+    <delete file="${test.build.dir}/test-udfs.jar"/> 
+    <jar jarfile="${test.build.dir}/test-udfs.jar">
+    	<fileset dir="${test.build.classes}" includes="**/udf/*.class"/>
+    </jar>
+  </target>
  
   <!-- Override jar target to specify main class and compiler stuff -->
 

Added: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionTask.java Fri Sep 19 03:19:22 2008
@@ -0,0 +1,63 @@
+/**
+ * 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.ql.exec;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.plan.FunctionWork;
+import org.apache.hadoop.hive.ql.plan.createFunctionDesc;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.hive.ql.exec.FunctionInfo.OperatorType;
+
+public class FunctionTask extends Task<FunctionWork> {
+  private static final long serialVersionUID = 1L;
+  private static final Log LOG = LogFactory.getLog("hive.ql.exec.FunctionTask");
+  
+  transient HiveConf conf;
+  
+  public void initialize(HiveConf conf) {
+    super.initialize(conf);
+    this.conf = conf;
+  }
+  
+  @Override
+  public int execute() {
+    createFunctionDesc createFunctionDesc = work.getCreateFunctionDesc();
+    if (createFunctionDesc != null) {
+      try {
+        Class<? extends UDF> udfClass = getUdfClass(createFunctionDesc);
+        FunctionRegistry.registerUDF(createFunctionDesc.getFunctionName(), udfClass,
+                                     OperatorType.PREFIX, false);
+        return 0;
+      } catch (ClassNotFoundException e) {
+        LOG.info("create function: " + StringUtils.stringifyException(e));
+        return 1;
+      }
+    }
+    return 0;
+  }
+
+  @SuppressWarnings("unchecked")
+  private Class<? extends UDF> getUdfClass(createFunctionDesc desc)
+      throws ClassNotFoundException {
+    return (Class<? extends UDF>) conf.getClassByName(desc.getClassName());
+  }
+
+}

Modified: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java?rev=697035&r1=697034&r2=697035&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java (original)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/exec/TaskFactory.java Fri Sep 19 03:19:22 2008
@@ -46,6 +46,7 @@
     taskvec.add(new taskTuple<moveWork>(moveWork.class, MoveTask.class));
     taskvec.add(new taskTuple<copyWork>(copyWork.class, CopyTask.class));
     taskvec.add(new taskTuple<DDLWork>(DDLWork.class, DDLTask.class));
+    taskvec.add(new taskTuple<FunctionWork>(FunctionWork.class, FunctionTask.class));
     taskvec.add(new taskTuple<explainWork>(explainWork.class, ExplainTask.class));
     // we are taking this out to allow us to instantiate either MapRedTask or
     // ExecDriver dynamically at run time based on configuration

Added: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/FunctionSemanticAnalyzer.java Fri Sep 19 03:19:22 2008
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.parse;
+
+import org.antlr.runtime.tree.CommonTree;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.plan.FunctionWork;
+import org.apache.hadoop.hive.ql.plan.createFunctionDesc;
+
+public class FunctionSemanticAnalyzer extends BaseSemanticAnalyzer {
+  private static final Log LOG =
+    LogFactory.getLog("hive.ql.parse.FunctionSemanticAnalyzer");
+  
+  public FunctionSemanticAnalyzer(HiveConf conf) throws SemanticException {
+    super(conf);
+  }
+  
+  public void analyze(CommonTree ast, Context ctx) throws SemanticException {
+    String functionName = ast.getChild(0).getText();
+    String className = unescapeSQLString(ast.getChild(1).getText());
+    createFunctionDesc desc = new createFunctionDesc(functionName, className);
+    rootTasks.add(TaskFactory.get(new FunctionWork(desc), conf));
+    LOG.info("analyze done");
+  }
+}

Modified: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g?rev=697035&r1=697034&r2=697035&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g (original)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/Hive.g Fri Sep 19 03:19:22 2008
@@ -97,6 +97,7 @@
 TOK_TABLELOCATION;
 TOK_TABLESAMPLE;
 TOK_TMP_FILE;
+TOK_CREATEFUNCTION;
 TOK_EXPLAIN;
 }
 
@@ -141,6 +142,7 @@
     | alterStatement
     | descStatement
     | showStatement
+    | createFunctionStatement
     ;
 
 createStatement
@@ -175,6 +177,11 @@
 showStatement
     : KW_SHOW KW_TABLES showStmtIdentifier?  -> ^(TOK_SHOWTABLES showStmtIdentifier?)
     ;
+    
+createFunctionStatement
+    : KW_CREATE KW_TEMPORARY KW_FUNCTION Identifier KW_AS StringLiteral
+    -> ^(TOK_CREATEFUNCTION Identifier StringLiteral)
+    ;
 
 showStmtIdentifier
     : Identifier
@@ -776,6 +783,8 @@
 KW_COLUMNS: 'COLUMNS';
 KW_RLIKE: 'RLIKE';
 KW_REGEXP: 'REGEXP';
+KW_TEMPORARY: 'TEMPORARY';
+KW_FUNCTION: 'FUNCTION';
 KW_EXPLAIN: 'EXPLAIN';
 KW_EXTENDED: 'EXTENDED';
 

Modified: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=697035&r1=697034&r2=697035&view=diff
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Fri Sep 19 03:19:22 2008
@@ -39,6 +39,8 @@
       case HiveParser.TOK_ALTERTABLE_RENAME:
       case HiveParser.TOK_SHOWTABLES: 
         return new DDLSemanticAnalyzer(conf);
+      case HiveParser.TOK_CREATEFUNCTION: 
+        return new FunctionSemanticAnalyzer(conf);
       default: return new SemanticAnalyzer(conf);
       }
     }

Added: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/FunctionWork.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/FunctionWork.java?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/FunctionWork.java (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/FunctionWork.java Fri Sep 19 03:19:22 2008
@@ -0,0 +1,38 @@
+/**
+ * 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.ql.plan;
+
+import java.io.Serializable;
+
+public class FunctionWork implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private createFunctionDesc  createFunctionDesc;
+  
+  public FunctionWork(createFunctionDesc createFunctionDesc) {
+    this.createFunctionDesc = createFunctionDesc;
+  }
+
+  public createFunctionDesc getCreateFunctionDesc() {
+    return createFunctionDesc;
+  }
+  public void setCreateFunctionDesc(createFunctionDesc createFunctionDesc) {
+    this.createFunctionDesc = createFunctionDesc;
+  }
+  
+}

Added: hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/java/org/apache/hadoop/hive/ql/plan/createFunctionDesc.java Fri Sep 19 03:19:22 2008
@@ -0,0 +1,53 @@
+/**
+ * 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.ql.plan;
+
+import java.io.Serializable;
+
+@explain(displayName="Create Function")
+public class createFunctionDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+  
+  private String functionName;
+  private String className;
+  
+  public createFunctionDesc(String functionName, String className) {
+    this.functionName = functionName;
+    this.className = className;
+  }
+
+  @explain(displayName="name")
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  public void setFunctionName(String functionName) {
+    this.functionName = functionName;
+  }
+
+  @explain(displayName="class")
+  public String getClassName() {
+    return className;
+  }
+
+  public void setClassName(String className) {
+    this.className = className;
+  }
+  
+}

Added: hadoop/core/trunk/src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/udf/UDFTestLength.java
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/udf/UDFTestLength.java?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/udf/UDFTestLength.java (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/test/org/apache/hadoop/hive/ql/udf/UDFTestLength.java Fri Sep 19 03:19:22 2008
@@ -0,0 +1,30 @@
+/**
+ * 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.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.UDF;
+
+/**
+ * A UDF for testing, which evaluates the length of a string.
+ */
+public class UDFTestLength extends UDF {
+  public Integer evaluate(String s) {
+    return s == null ? null : s.length();
+  }
+}
\ No newline at end of file

Added: hadoop/core/trunk/src/contrib/hive/ql/src/test/queries/clientpositive/udf2.q
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/test/queries/clientpositive/udf2.q?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/test/queries/clientpositive/udf2.q (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/test/queries/clientpositive/udf2.q Fri Sep 19 03:19:22 2008
@@ -0,0 +1,10 @@
+EXPLAIN
+CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength';
+
+CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength';
+
+CREATE TABLE dest1(len INT);
+
+FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength(src.value);
+
+SELECT dest1.* FROM dest1;

Added: hadoop/core/trunk/src/contrib/hive/ql/src/test/results/clientpositive/udf2.q.out
URL: http://svn.apache.org/viewvc/hadoop/core/trunk/src/contrib/hive/ql/src/test/results/clientpositive/udf2.q.out?rev=697035&view=auto
==============================================================================
--- hadoop/core/trunk/src/contrib/hive/ql/src/test/results/clientpositive/udf2.q.out (added)
+++ hadoop/core/trunk/src/contrib/hive/ql/src/test/results/clientpositive/udf2.q.out Fri Sep 19 03:19:22 2008
@@ -0,0 +1,509 @@
+ABSTRACT SYNTAX TREE:	
+  (TOK_CREATEFUNCTION testlength 'org.apache.hadoop.hive.ql.udf.UDFTestLength')	
+	
+STAGE DEPENDENCIES:	
+  Stage-0 is a root stage	
+	
+STAGE PLANS:	
+  Stage: Stage-0	
+	
+7	
+6	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+5	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+7	
+6	
+6	
+5	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+6	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+6	
+5	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+5	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+6	
+5	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+6	
+6	
+7	
+6	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+5	
+7	
+7	
+7	
+7	
+7	
+6	
+6	
+7	
+6	
+6	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+6	
+6	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+6	
+7	
+6	
+6	
+7	
+6	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+5	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+5	
+6	
+7	
+7	
+7	
+6	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+5	
+6	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+6	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+5	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+6	
+6	
+6	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+7	
+6	
+7	
+7	
+7	
+7	
+6