[SQL] Update Hive test harness for Hive 12 and 13

As part of the upgrade I also copy the newest version of the query tests, and whitelist a bunch of new ones that are now passing.

Author: Michael Armbrust <michael@databricks.com>

Closes #2936 from marmbrus/fix13tests and squashes the following commits:

d9cbdab [Michael Armbrust] Remove user specific tests
65801cd [Michael Armbrust] style and rat
8f6b09a [Michael Armbrust] Update test harness to work with both Hive 12 and 13.
f044843 [Michael Armbrust] Update Hive query tests and golden files to 0.13
This commit is contained in:
Michael Armbrust 2014-10-24 18:36:35 -07:00 committed by Josh Rosen
parent 898b22ab1f
commit 3a845d3c04
8166 changed files with 38301 additions and 47481 deletions

View file

@ -48,6 +48,7 @@ sbt-launch-lib.bash
plugins.sbt
work
.*\.q
.*\.qv
golden
test.out/*
.*iml

View file

@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS
# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled.
# This must be a single argument, as it is.
if [ -n "$_RUN_SQL_TESTS" ]; then
SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0"
SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive"
fi
if [ -n "$_SQL_TESTS_ONLY" ]; then

View file

@ -253,7 +253,11 @@ object Hive {
|import org.apache.spark.sql.hive._
|import org.apache.spark.sql.hive.test.TestHive._
|import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin,
cleanupCommands in console := "sparkContext.stop()"
cleanupCommands in console := "sparkContext.stop()",
// Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
// in order to generate golden files. This is only required for developers who are adding new
// new query tests.
fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") }
)
}

View file

@ -52,6 +52,8 @@ object HiveTypeCoercion {
*/
trait HiveTypeCoercion {
import HiveTypeCoercion._
val typeCoercionRules =
PropagateTypes ::
ConvertNaNs ::
@ -340,6 +342,13 @@ trait HiveTypeCoercion {
// Skip nodes who's children have not been resolved yet.
case e if !e.childrenResolved => e
case a @ CreateArray(children) if !a.resolved =>
val commonType = a.childTypes.reduce(
(a,b) =>
findTightestCommonType(a,b).getOrElse(StringType))
CreateArray(
children.map(c => if (c.dataType == commonType) c else Cast(c, commonType)))
// Promote SUM, SUM DISTINCT and AVERAGE to largest types to prevent overflows.
case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest.
case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType))
@ -356,6 +365,10 @@ trait HiveTypeCoercion {
Average(Cast(e, LongType))
case Average(e @ FractionalType()) if e.dataType != DoubleType =>
Average(Cast(e, DoubleType))
// Hive lets you do aggregation of timestamps... for some reason
case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType))
case Average(e @ TimestampType()) => Average(Cast(e, DoubleType))
}
}

View file

@ -101,3 +101,28 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio
override def toString = s"$child.$fieldName"
}
/**
* Returns an Array containing the evaluation of all children expressions.
*/
case class CreateArray(children: Seq[Expression]) extends Expression {
override type EvaluatedType = Any
lazy val childTypes = children.map(_.dataType).distinct
override lazy val resolved =
childrenResolved && childTypes.size <= 1
override def dataType: DataType = {
assert(resolved, s"Invalid dataType of mixed ArrayType ${childTypes.mkString(",")}")
ArrayType(childTypes.headOption.getOrElse(NullType))
}
override def nullable: Boolean = false
override def eval(input: Row): Any = {
children.map(_.eval(input))
}
override def toString = s"Array(${children.mkString(",")})"
}

View file

@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.catalyst.types.DataType
import org.apache.spark.sql.columnar.InMemoryRelation
import org.apache.spark.sql.execution.{SparkStrategies, _}
import org.apache.spark.sql.json._
import org.apache.spark.sql.parquet.ParquetRelation

View file

@ -23,6 +23,7 @@ import java.util.{Locale, TimeZone}
import org.scalatest.BeforeAndAfter
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.sql.hive.test.TestHive
/**
@ -102,6 +103,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf5",
"udf_java_method",
"create_merge_compressed",
"database_location",
"database_properties",
// DFS commands
"symlink_text_input_format",
@ -135,6 +138,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"stats20",
"alter_merge_stats",
"columnstats.*",
"annotate_stats.*",
"database_drop",
"index_serde",
// Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong.
@ -211,8 +217,20 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"describe_comment_indent",
// Limit clause without a ordering, which causes failure.
"orc_predicate_pushdown"
)
"orc_predicate_pushdown",
// Requires precision decimal support:
"decimal_1",
"udf_pmod",
"udf_when",
"udf_case",
"udf_to_double",
"udf_to_float",
// Needs constant object inspectors
"udf_round",
"udf7"
) ++ HiveShim.compatibilityBlackList
/**
* The set of tests that are believed to be working in catalyst. Tests not on whiteList or
@ -220,23 +238,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
*/
override def whiteList = Seq(
"add_part_exist",
"dynamic_partition_skip_default",
"infer_bucket_sort_dyn_part",
"load_dyn_part1",
"load_dyn_part2",
"load_dyn_part3",
"load_dyn_part4",
"load_dyn_part5",
"load_dyn_part6",
"load_dyn_part7",
"load_dyn_part8",
"load_dyn_part9",
"load_dyn_part10",
"load_dyn_part11",
"load_dyn_part12",
"load_dyn_part13",
"load_dyn_part14",
"load_dyn_part14_win",
"add_part_multiple",
"add_partition_no_whitelist",
"add_partition_with_whitelist",
@ -256,6 +257,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"alter_varchar2",
"alter_view_as_select",
"ambiguous_col",
"annotate_stats_join",
"annotate_stats_limit",
"annotate_stats_part",
"annotate_stats_table",
"annotate_stats_union",
"auto_join0",
"auto_join1",
"auto_join10",
@ -299,6 +305,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"auto_sortmerge_join_13",
"auto_sortmerge_join_14",
"auto_sortmerge_join_15",
"auto_sortmerge_join_16",
"auto_sortmerge_join_2",
"auto_sortmerge_join_3",
"auto_sortmerge_join_4",
@ -340,7 +347,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"create_skewed_table1",
"create_struct_table",
"cross_join",
"cross_product_check_1",
"cross_product_check_2",
"ct_case_insensitive",
"database_drop",
"database_location",
"database_properties",
"date_2",
@ -360,8 +370,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"diff_part_input_formats",
"disable_file_format_check",
"disallow_incompatible_type_change_off",
"distinct_stats",
"drop_database_removes_partition_dirs",
"drop_function",
"drop_index",
"drop_index_removes_partition_dirs",
"drop_multi_partitions",
"drop_partitions_filter",
"drop_partitions_filter2",
@ -369,23 +382,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"drop_partitions_ignore_protection",
"drop_table",
"drop_table2",
"drop_table_removes_partition_dirs",
"drop_view",
"dynamic_partition_skip_default",
"escape_clusterby1",
"escape_distributeby1",
"escape_orderby1",
"escape_sortby1",
"explain_rearrange",
"fetch_aggregation",
"fileformat_mix",
"fileformat_sequencefile",
"fileformat_text",
"filter_join_breaktask",
"filter_join_breaktask2",
"groupby1",
"groupby11",
"groupby12",
"groupby1_limit",
"groupby1_map",
"groupby1_map_nomap",
"groupby1_map_skew",
"groupby1_noskew",
"groupby2",
"groupby2_limit",
"groupby2_map",
"groupby2_map_skew",
"groupby2_noskew",
@ -406,6 +426,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"groupby7_map_multi_single_reducer",
"groupby7_map_skew",
"groupby7_noskew",
"groupby7_noskew_multi_single_reducer",
"groupby8",
"groupby8_map",
"groupby8_map_skew",
@ -432,6 +453,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"groupby_sort_test_1",
"having",
"implicit_cast1",
"index_serde",
"infer_bucket_sort_dyn_part",
"innerjoin",
"inoutdriver",
"input",
@ -502,7 +525,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"join17",
"join18",
"join19",
"join_1to1",
"join2",
"join20",
"join21",
@ -534,6 +556,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"join7",
"join8",
"join9",
"join_1to1",
"join_array",
"join_casesensitive",
"join_empty",
@ -557,7 +580,21 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"literal_double",
"literal_ints",
"literal_string",
"load_dyn_part1",
"load_dyn_part10",
"load_dyn_part11",
"load_dyn_part12",
"load_dyn_part13",
"load_dyn_part14",
"load_dyn_part14_win",
"load_dyn_part2",
"load_dyn_part3",
"load_dyn_part4",
"load_dyn_part5",
"load_dyn_part6",
"load_dyn_part7",
"load_dyn_part8",
"load_dyn_part9",
"load_file_with_space_in_the_name",
"loadpart1",
"louter_join_ppr",
@ -578,13 +615,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"merge1",
"merge2",
"mergejoins",
"multigroupby_singlemr",
"multiMapJoin1",
"multiMapJoin2",
"multi_insert_gby",
"multi_insert_gby3",
"multi_insert_lateral_view",
"multi_join_union",
"multiMapJoin1",
"multiMapJoin2",
"multigroupby_singlemr",
"noalias_subq1",
"nomore_ambiguous_table_col",
"nonblock_op_deduplicate",
@ -607,10 +644,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"outer_join_ppr",
"parallel",
"parenthesis_star_by",
"partcols1",
"part_inherit_tbl_props",
"part_inherit_tbl_props_empty",
"part_inherit_tbl_props_with_star",
"partcols1",
"partition_date",
"partition_schema1",
"partition_serde_format",
@ -641,7 +678,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"ppd_outer_join5",
"ppd_random",
"ppd_repeated_alias",
"ppd_transform",
"ppd_udf_col",
"ppd_union",
"ppr_allchildsarenull",
@ -674,15 +710,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"serde_regex",
"serde_reported_schema",
"set_variable_sub",
"show_create_table_partitioned",
"show_create_table_delimited",
"show_columns",
"show_create_table_alter",
"show_create_table_view",
"show_create_table_serde",
"show_create_table_db_table",
"show_create_table_delimited",
"show_create_table_does_not_exist",
"show_create_table_index",
"show_columns",
"show_create_table_partitioned",
"show_create_table_serde",
"show_create_table_view",
"show_describe_func_quotes",
"show_functions",
"show_partitions",
@ -738,12 +774,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udaf_covar_pop",
"udaf_covar_samp",
"udaf_histogram_numeric",
"udf_10_trims",
"udf2",
"udf6",
"udf7",
"udf8",
"udf9",
"udf_10_trims",
"udf_E",
"udf_PI",
"udf_abs",
"udf_acos",
"udf_add",
@ -774,14 +812,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_cos",
"udf_count",
"udf_date_add",
"udf_datediff",
"udf_date_sub",
"udf_datediff",
"udf_day",
"udf_dayofmonth",
"udf_degrees",
"udf_div",
"udf_double",
"udf_E",
"udf_elt",
"udf_equal",
"udf_exp",
@ -826,7 +863,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_nvl",
"udf_or",
"udf_parse_url",
"udf_PI",
"udf_pmod",
"udf_positive",
"udf_pow",

View file

@ -46,7 +46,6 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs
import org.apache.spark.sql.execution.QueryExecutionException
import org.apache.spark.sql.execution.{Command => PhysicalCommand}
import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand
import org.apache.spark.sql.hive.HiveShim
/**
* DEPRECATED: Use HiveContext instead.
@ -230,7 +229,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
* set() or a SET command inside sql() will be set in the SQLConf *as well as*
* in the HiveConf.
*/
@transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState])
@transient lazy val hiveconf = new HiveConf(classOf[SessionState])
@transient protected[hive] lazy val sessionState = {
val ss = new SessionState(hiveconf)
setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf.

View file

@ -26,7 +26,6 @@ import org.apache.hadoop.{io => hadoopIo}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.hive.HiveShim
/* Implicit conversions */
import scala.collection.JavaConversions._

View file

@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.hive.HiveShim
import org.apache.spark.util.Utils
/* Implicit conversions */

View file

@ -251,6 +251,8 @@ private[hive] object HiveQl {
s"""
|Unsupported language features in query: $sql
|${dumpTree(getAst(sql))}
|$e
|${e.getStackTrace.head}
""".stripMargin)
}
}
@ -329,6 +331,7 @@ private[hive] object HiveQl {
case Token("TOK_SMALLINT", Nil) => ShortType
case Token("TOK_BOOLEAN", Nil) => BooleanType
case Token("TOK_STRING", Nil) => StringType
case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType
case Token("TOK_FLOAT", Nil) => FloatType
case Token("TOK_DOUBLE", Nil) => DoubleType
case Token("TOK_DATE", Nil) => DateType
@ -854,9 +857,11 @@ private[hive] object HiveQl {
HiveParser.Number,
HiveParser.TinyintLiteral,
HiveParser.SmallintLiteral,
HiveParser.BigintLiteral)
HiveParser.BigintLiteral,
HiveParser.DecimalLiteral)
/* Case insensitive matches */
val ARRAY = "(?i)ARRAY".r
val COUNT = "(?i)COUNT".r
val AVG = "(?i)AVG".r
val SUM = "(?i)SUM".r
@ -917,7 +922,9 @@ private[hive] object HiveQl {
/* Casts */
case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) =>
Cast(nodeToExpr(arg), StringType)
case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) =>
case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) =>
Cast(nodeToExpr(arg), StringType)
case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) =>
Cast(nodeToExpr(arg), StringType)
case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) =>
Cast(nodeToExpr(arg), IntegerType)
@ -1009,6 +1016,8 @@ private[hive] object HiveQl {
GetItem(nodeToExpr(child), nodeToExpr(ordinal))
/* Other functions */
case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) =>
CreateArray(children.map(nodeToExpr))
case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand
case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) =>
Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType))
@ -1042,10 +1051,10 @@ private[hive] object HiveQl {
} else if (ast.getText.endsWith("Y")) {
// Literal tinyint.
v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType)
} else if (ast.getText.endsWith("BD")) {
} else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) {
// Literal decimal
val strVal = ast.getText.substring(0, ast.getText.length() - 2)
BigDecimal(strVal)
val strVal = ast.getText.stripSuffix("D").stripSuffix("B")
v = Literal(BigDecimal(strVal))
} else {
v = Literal(ast.getText.toDouble, DoubleType)
v = Literal(ast.getText.toLong, LongType)
@ -1056,7 +1065,7 @@ private[hive] object HiveQl {
}
if (v == null) {
sys.error(s"Failed to parse number ${ast.getText}")
sys.error(s"Failed to parse number '${ast.getText}'.")
} else {
v
}

View file

@ -34,7 +34,6 @@ import org.apache.spark.SerializableWritable
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.hive.HiveShim
/**
* A trait for subclasses that handle table scans.

View file

@ -20,6 +20,9 @@ package org.apache.spark.sql.hive.test
import java.io.File
import java.util.{Set => JavaSet}
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.session.SessionState
import scala.collection.mutable
import scala.language.implicitConversions
@ -119,7 +122,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
if (cmd.toUpperCase contains "LOAD DATA") {
val testDataLocation =
hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath)
cmd.replaceAll("\\.\\.", testDataLocation)
cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/")
} else {
cmd
}
@ -417,6 +420,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
FunctionRegistry.unregisterTemporaryUDF(udfName)
}
// Some tests corrupt this value on purpose, which breaks the RESET call below.
hiveconf.set("fs.default.name", new File(".").toURI.toString)
// It is important that we RESET first as broken hooks that might have been set could break
// other sql exec here.
runSqlHive("RESET")

View file

@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.api.java
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD}
import org.apache.spark.sql.SQLConf
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.hive.{HiveContext, HiveQl}
/**
* The entry point for executing Spark SQL queries from a Java program.
*/
class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(sparkContext) {
class JavaHiveContext(sqlContext: SQLContext) extends JavaSQLContext(sqlContext) {
override val sqlContext = new HiveContext(sparkContext)
def this(sparkContext: JavaSparkContext) = this(new HiveContext(sparkContext))
override def sql(sqlText: String): JavaSchemaRDD = {
// TODO: Create a framework for registering parsers instead of just hardcoding if statements.

View file

@ -45,7 +45,7 @@ case class DescribeHiveTableCommand(
lazy val hiveString: Seq[String] = sideEffectResult.map {
case Row(name: String, dataType: String, comment) =>
Seq(name, dataType,
Option(comment.asInstanceOf[String]).getOrElse(HiveShim.getEmptyCommentsFieldValue))
Option(comment.asInstanceOf[String]).getOrElse(""))
.map(s => String.format(s"%-20s", s))
.mkString("\t")
}

View file

@ -19,6 +19,12 @@
<configuration>
<property>
<name>hive.in.test</name>
<value>true</value>
<description>Internal marker for test. Used for masking env-dependent values</description>
</property>
<!-- Hive Configuration can either be stored in this file or in the hadoop configuration files -->
<!-- that are implied by Hadoop setup variables. -->
<!-- Aside from Hadoop setup variables - this file is provided as a convenience so that Hive -->
@ -28,7 +34,7 @@
<!-- Hive Execution Parameters -->
<property>
<name>hadoop.tmp.dir</name>
<value>${build.dir.hive}/test/hadoop-${user.name}</value>
<value>${test.tmp.dir}/hadoop-tmp</value>
<description>A base for other temporary directories.</description>
</property>
@ -42,22 +48,27 @@
<property>
<name>hive.exec.scratchdir</name>
<value>${build.dir}/scratchdir</value>
<value>${test.tmp.dir}/scratchdir</value>
<description>Scratch space for Hive jobs</description>
</property>
<property>
<name>hive.exec.local.scratchdir</name>
<value>${build.dir}/localscratchdir/</value>
<value>${test.tmp.dir}/localscratchdir/</value>
<description>Local scratch space for Hive jobs</description>
</property>
<property>
<name>javax.jdo.option.ConnectionURL</name>
<!-- note: variable substituion not working here because it's loaded by jdo, not Hive -->
<value>jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true</value>
<value>jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true</value>
</property>
<property>
<name>hive.stats.dbconnectionstring</name>
<value>jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true</value>
</property>
<property>
<name>javax.jdo.option.ConnectionDriverName</name>
<value>org.apache.derby.jdbc.EmbeddedDriver</value>
@ -82,7 +93,7 @@
<property>
<name>hive.metastore.metadb.dir</name>
<value>file://${build.dir}/test/data/metadb/</value>
<value>file://${test.tmp.dir}/metadb/</value>
<description>
Required by metastore server or if the uris argument below is not supplied
</description>
@ -90,32 +101,19 @@
<property>
<name>test.log.dir</name>
<value>${build.dir}/test/logs</value>
<description></description>
</property>
<property>
<name>test.src.dir</name>
<value>file://${build.dir}/src/test</value>
<value>${test.tmp.dir}/log/</value>
<description></description>
</property>
<property>
<name>test.data.files</name>
<value>${user.dir}/../data/files</value>
<description></description>
</property>
<property>
<name>test.query.file1</name>
<value>file://${user.dir}/../ql/src/test/org/apache/hadoop/hive/ql/input2.q</value>
<value></value>
<value>${hive.root}/data/files</value>
<description></description>
</property>
<property>
<name>hive.jar.path</name>
<value>${build.dir.hive}/ql/hive-exec-${version}.jar</value>
<value>${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar</value>
<description></description>
</property>
@ -127,7 +125,7 @@
<property>
<name>hive.querylog.location</name>
<value>${build.dir}/tmp</value>
<value>${test.tmp.dir}/tmp</value>
<description>Location of the structured hive logs</description>
</property>
@ -143,18 +141,25 @@
<description>Post Execute Hook for Tests</description>
</property>
<property>
<name>hive.task.progress</name>
<value>false</value>
<description>Track progress of a task</description>
</property>
<property>
<name>hive.support.concurrency</name>
<value>true</value>
<description>Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks.</description>
</property>
<property>
<key>hive.unlock.numretries</key>
<value>2</value>
<description>The number of times you want to retry to do one unlock</description>
</property>
<property>
<key>hive.lock.sleep.between.retries</key>
<value>2</value>
<description>The sleep time (in seconds) between various retries</description>
</property>
<property>
<name>fs.pfile.impl</name>
<value>org.apache.hadoop.fs.ProxyLocalFileSystem</value>
@ -194,4 +199,21 @@
<description>The default SerDe hive will use for the rcfile format</description>
</property>
<property>
<name>hive.stats.dbclass</name>
<value>jdbc:derby</value>
<description>The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported</description>
</property>
<property>
<name>hive.stats.key.prefix.reserve.length</name>
<value>0</value>
</property>
<property>
<name>hive.conf.restricted.list</name>
<value>dummy.config.value</value>
<description>Using dummy config value above because you cannot override config with empty value</description>
</property>
</configuration>

View file

@ -0,0 +1,15 @@
+-----+-------+
| id | name |
+-----+-------+
| 1 | aaa |
| 2 | bbb |
| 3 | ccc |
| 4 | ddd |
| 5 | eee |
+-----+-------+
+-------+-----+
| name | id |
+-------+-----+
| aaa | 1 |
| bbb | 2 |
+-------+-----+

View file

@ -0,0 +1,2 @@
true|10|100|1000|10000|4.0|20.0|2.2222|1969-12-31 15:59:58.174|1970-01-01 00:00:00|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"}
true|20|200|2000|20000|8.0|40.0|4.2222|1970-12-31 15:59:58.174|1971-01-01 00:00:00|||k3:v3,k4:v4|200,300|{20, "bar"}

View file

@ -0,0 +1,2 @@
true|10|100|1000|10000|4.0|20.0|4.2222|1969-12-31 15:59:58.174|1970-01-01|string|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"}
false|20|200|2000|20000|8.0|40.0|2.2222|1970-12-31 15:59:58.174|1971-01-01|abcd|world|world|k3:v3,k4:v4|200,300|{20, "bar"}

Binary file not shown.

View file

@ -0,0 +1 @@
47.3224 2923.29346

View file

@ -1,3 +1,3 @@
\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N
-1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N
1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123
\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N
-1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N\N
1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123abc123X'01FF'

View file

@ -0,0 +1,18 @@
55.33
44.2
435.33
324.33
324.33
44.2
55.3
55.3
0.0
66.4
23.22
-87.2
33.44
55.3
435.331
-0.342

View file

@ -0,0 +1,4 @@
31|sales
33|engineering
34|clerical
35|marketing

View file

@ -0,0 +1,6 @@
Rafferty|31
Jones|33
Steinberg|33
Robinson|34
Smith|34
John|

View file

@ -0,0 +1 @@
{"partitions":[],"table":"{\"1\":{\"str\":\"j1_41\"},\"2\":{\"str\":\"default\"},\"3\":{\"str\":\"johndee\"},\"4\":{\"i32\":1371900915},\"5\":{\"i32\":0},\"6\":{\"i32\":0},\"7\":{\"rec\":{\"1\":{\"lst\":[\"rec\",2,{\"1\":{\"str\":\"a\"},\"2\":{\"str\":\"string\"}},{\"1\":{\"str\":\"b\"},\"2\":{\"str\":\"int\"}}]},\"2\":{\"str\":\"hdfs://hivebase01:8020/user/hive/warehouse/j1_41\"},\"3\":{\"str\":\"org.apache.hadoop.mapred.TextInputFormat\"},\"4\":{\"str\":\"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat\"},\"5\":{\"tf\":0},\"6\":{\"i32\":-1},\"7\":{\"rec\":{\"2\":{\"str\":\"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\"},\"3\":{\"map\":[\"str\",\"str\",2,{\"serialization.format\":\",\",\"field.delim\":\",\"}]}}},\"8\":{\"lst\":[\"str\",0]},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",0,{}]}}},\"8\":{\"lst\":[\"rec\",0]},\"9\":{\"map\":[\"str\",\"str\",1,{\"transient_lastDdlTime\":\"1371900931\"}]},\"12\":{\"str\":\"MANAGED_TABLE\"}}","version":"0.1"}

View file

@ -0,0 +1,2 @@
johndee,1
burks,2

View file

@ -0,0 +1,3 @@
12 jason
13 steven
15 joe

View file

@ -0,0 +1,8 @@
name message 0
steven hive 1
dave oozie 2
xifa phd 3
chuan hadoop 4
shanyu senior 5
footer1 footer1 0
footer2 0

View file

@ -0,0 +1,8 @@
name message 0
steven2 hive 11
dave2 oozie 12
xifa2 phd 13
chuan2 hadoop 14
shanyu2 senior 15
footer1 footer1 0
footer2 0

View file

@ -0,0 +1,4 @@
name message 0
david3 oozie 22
footer1 footer1 0
footer2 0

View file

@ -0,0 +1,8 @@
name message 0
steven hive 1
dave oozie 2
xifa phd 3
chuan hadoop 4
shanyu senior 5
footer1 footer1 0
footer2 0

View file

@ -0,0 +1,8 @@
name message 0
steven2 hive 11
dave2 oozie 12
xifa2 phd 13
chuan2 hadoop 14
shanyu2 senior 15
footer1 footer1 0
footer2 0

View file

@ -0,0 +1,4 @@
name message 0
david3 oozie 22
footer1 footer1 0
footer2 0

View file

@ -0,0 +1,7 @@
a b c d e f g
a b c d e f g
a b c d e f g
d e f g
a b c d
a e f g
a d g

Binary file not shown.

View file

@ -0,0 +1,27 @@
-4400 4400
1E+99 0
1E-99 0
0 0
10 10
23232.23435 2
2389432.23752 3
2389432.2375 4
10.73433 5
0.333 0
-0.3 0
-0.333 0
1.0 1
2 2
3.14 3
-1.12 -1
-1.122 -11
1.12 1
1.122 1
124.00 124
125.2 125
-1255.49 -1255
3.14 3
3.140 4
0.9999999999999999999999999 1
-1234567890.1234567890 -1234567890
1234567890.1234567800 1234567890

View file

@ -0,0 +1,8 @@
OH|31|43201|2001
IO|32|43202|2001
CA|35|43809|2001
FL|33|54342|2001
UT|35||2001
CA|35|43809|2001
|34|40000|
FL|33|54342|2001

View file

@ -0,0 +1 @@
1|Garçu Kôkaku kidôtai

View file

@ -1,100 +1,100 @@
1CelesteBrowning959-3763 Nec, Av.Ca
2RisaYangP.O. Box 292, 8229 Porttitor RoadOr
3VenusSuttonAp #962-8021 Egestas Rd.Ca
4GretchenHarrisonP.O. Box 636, 8734 Magna AvenueOr
5LaniIrwinAp #441-5911 Iaculis, AveCa
6VeraGeorge409-1555 Vel, AveOr
7JessicaMalone286-9779 Aliquam RoadCa
8AnnChapmanAp #504-3915 Placerat RoadOr
9NigelBartlettAp #185-385 Diam StreetCa
10AzaliaJennings5772 Diam St.Or
11PrestonCannonAp #527-8769 Nunc AvenueCa
12AllistairVasquez2562 Odio. St.Or
13ReedHayes5190 Elit StreetCa
14ElaineBarronP.O. Box 840, 8860 Sodales Av.Or
15LydiaHoodP.O. Box 698, 5666 Semper RoadCa
16VanceMaxwell298-3313 Malesuada RoadOr
17KeikoDeleonP.O. Box 732, 5921 Massa. Av.Ca
18DolanKaneAp #906-3606 Ut Rd.Or
19MerrittPerkinsP.O. Box 228, 7090 Egestas StreetCa
20CaseySalazar506-5065 Ut St.Or
21SamsonNoel1370 Ultrices, RoadCa
22ByronWalkerP.O. Box 386, 8324 Tellus AveOr
23PiperSingletonAp #500-3561 Primis St.Ca
24RiaMckinney3080 Dui Rd.Or
25RahimStanley559-9016 Nascetur StreetCa
26ChloeSteeleP.O. Box 766, 1628 Elit StreetOr
27PalomaWardAp #390-3042 Ipsum Rd.Ca
28RoaryShermanAp #409-6549 Metus St.Or
29CalvinBuckner6378 Diam AvenueCa
30CamilleGoodAp #113-8659 Suspendisse St.Or
31SteelAyala5518 Justo St.Ca
32JosiahGilbertAp #149-6651 At, Av.Or
33HamiltonCruz4620 Tellus. AveCa
34ScarletSantos586-1785 Velit. Av.Or
35LewisMcintyre629-6419 Ac Rd.Ca
36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.Or
37VelmaHaley1377 At Rd.Ca
38TatumJennings829-7432 Posuere, RoadOr
39BritanniEaton8811 Morbi StreetCa
40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.Or
41KareemAyala2706 Ridiculus StreetCa
42MaiteRush7592 Neque RoadOr
43SigneVelasquezAp #868-3039 Eget St.Ca
44ZoritaCamachoP.O. Box 651, 3340 Quis Av.Or
45GlennaCurtis953-7965 Enim AveCa
46QuinCortez4898 Ridiculus St.Or
47TalonDaltonP.O. Box 408, 7597 Integer Rd.Ca
48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.Or
49VernonReyesP.O. Box 971, 7009 Vulputate StreetCa
50TallulahHeathP.O. Box 865, 3697 Dis AveOr
51CiaranOlson2721 Et St.Ca
52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.Or
53QuinnRiceAp #647-6627 Tristique AvenueCa
54WyattPickettAp #128-3130 Vel, Rd.Or
55EmeraldCopeland857-5119 Turpis Rd.Ca
56JonasQuinnAp #441-7183 Ligula. StreetOr
57WillaBerg6672 Velit AveCa
58MalikLee998-9208 In StreetOr
59CallieMedina1620 Dui. Rd.Ca
60LukeMasonP.O. Box 143, 2070 Augue Rd.Or
61ShafiraEstrada8824 Ante StreetCa
62ElizabethRutledge315-6510 Sit St.Or
63PandoraLevine357-3596 Nibh. AveCa
64HilelPrince845-1229 Sociosqu Rd.Or
65RinahTorresAp #492-9328 At St.Ca
66YaelHobbsP.O. Box 477, 3896 In StreetOr
67NevadaNashP.O. Box 251, 1914 Tincidunt RoadCa
68MarnyHuffP.O. Box 818, 6086 Ultricies St.Or
69KimberleyMilesAp #893-3685 In RoadCa
70DuncanFullerAp #197-5216 Iaculis StreetOr
71YardleyLeblancP.O. Box 938, 1278 Sit AveCa
72HamishBrewerAp #854-781 Quisque St.Or
73PetraMoon453-6609 Curabitur StreetCa
74ReeseEstradaAp #382-3313 Malesuada St.Or
75GageHiggins7443 Eu StreetCa
76ZacheryCamachoAp #795-4143 Quam. St.Or
77KellyGarnerP.O. Box 895, 2843 Cras Rd.Ca
78HanaeCarr9440 Amet St.Or
79AnnAlston884-7948 Dictum RoadCa
80ChancellorCobbP.O. Box 889, 5978 Ac AvenueOr
81DorothyHarrell6974 Tristique AveCa
82VaughanLeon1610 Luctus Av.Or
83WynneJimenez321-9171 Felis. AvenueCa
84WillaMendoza489-182 Sed Av.Or
85CamdenGoodwin4579 Ante St.Ca
86IfeomaFrenchP.O. Box 160, 8769 Integer RoadOr
87RamonaStrong1666 Ridiculus AvenueCa
88BrettRamosAp #579-9879 Et, RoadOr
89UllaGray595-7066 Malesuada RoadCa
90KevynMccallP.O. Box 968, 1420 Aenean AvenueOr
91GenevieveWilkins908 Turpis. StreetCa
92ThaneOneil6766 Lectus St.Or
93MarikoClineP.O. Box 329, 5375 Ac St.Ca
94LaelMclean500-7010 Sit St.Or
95WinifredHopperAp #140-8982 Velit AvenueCa
96RafaelEnglandP.O. Box 405, 7857 Eget Av.Or
97DanaCarter814-601 Purus. Av.Ca
98JulietBattleAp #535-1965 Cursus St.Or
99WynterVincent626-8492 Mollis AvenueCa
100WangMitchell4023 Lacinia. AveOr
1CelesteBrowning959-3763 Nec, Av.100.002011-03-12 15:20:00Ca
2RisaYangP.O. Box 292, 8229 Porttitor Road200.002011-03-12 15:20:00Or
3VenusSuttonAp #962-8021 Egestas Rd.300.002011-03-12 15:20:00Ca
4GretchenHarrisonP.O. Box 636, 8734 Magna Avenue400.002011-03-12 15:20:00Or
5LaniIrwinAp #441-5911 Iaculis, Ave500.002011-03-12 15:20:00Ca
6VeraGeorge409-1555 Vel, Ave600.002011-03-12 15:20:00Or
7JessicaMalone286-9779 Aliquam Road700.002011-03-12 15:20:00Ca
8AnnChapmanAp #504-3915 Placerat Road800.002011-03-12 15:20:00Or
9NigelBartlettAp #185-385 Diam Street900.002011-03-12 15:20:00Ca
10AzaliaJennings5772 Diam St.100.002011-03-12 15:20:00Or
11PrestonCannonAp #527-8769 Nunc Avenue100.002011-03-12 15:20:00Ca
12AllistairVasquez2562 Odio. St.100.002011-03-12 15:20:00Or
13ReedHayes5190 Elit Street100.002011-03-12 15:20:00Ca
14ElaineBarronP.O. Box 840, 8860 Sodales Av.100.002011-03-12 15:20:00Or
15LydiaHoodP.O. Box 698, 5666 Semper Road100.002011-03-12 15:20:00Ca
16VanceMaxwell298-3313 Malesuada Road100.002011-03-12 15:20:00Or
17KeikoDeleonP.O. Box 732, 5921 Massa. Av.100.002011-03-12 15:20:00Ca
18DolanKaneAp #906-3606 Ut Rd.100.002011-03-12 15:20:00Or
19MerrittPerkinsP.O. Box 228, 7090 Egestas Street100.002011-03-12 15:20:00Ca
20CaseySalazar506-5065 Ut St.200.002011-03-12 15:20:00Or
21SamsonNoel1370 Ultrices, Road200.002012-03-12 15:20:00Ca
22ByronWalkerP.O. Box 386, 8324 Tellus Ave200.002012-03-12 15:20:00Or
23PiperSingletonAp #500-3561 Primis St.200.002012-03-12 15:20:00Ca
24RiaMckinney3080 Dui Rd.200.002012-03-12 15:20:00Or
25RahimStanley559-9016 Nascetur Street200.002012-03-12 15:20:00Ca
26ChloeSteeleP.O. Box 766, 1628 Elit Street200.002012-03-12 15:20:00Or
27PalomaWardAp #390-3042 Ipsum Rd.200.002012-03-12 15:20:00Ca
28RoaryShermanAp #409-6549 Metus St.200.002012-03-12 15:20:00Or
29CalvinBuckner6378 Diam Avenue200.002012-03-12 15:20:00Ca
30CamilleGoodAp #113-8659 Suspendisse St.300.002012-03-12 15:20:00Or
31SteelAyala5518 Justo St.300.002012-03-12 15:20:00Ca
32JosiahGilbertAp #149-6651 At, Av.300.002012-03-12 15:20:00Or
33HamiltonCruz4620 Tellus. Ave300.002012-03-12 15:20:00Ca
34ScarletSantos586-1785 Velit. Av.300.002012-03-12 15:20:00Or
35LewisMcintyre629-6419 Ac Rd.300.002012-03-12 15:20:00Ca
36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.300.002012-03-12 15:20:00Or
37VelmaHaley1377 At Rd.300.002012-03-12 15:20:00Ca
38TatumJennings829-7432 Posuere, Road300.002012-03-12 15:20:00Or
39BritanniEaton8811 Morbi Street300.002012-03-12 15:20:00Ca
40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.400.002012-03-12 15:20:00Or
41KareemAyala2706 Ridiculus Street400.002013-03-12 15:20:00Ca
42MaiteRush7592 Neque Road400.002013-03-12 15:20:00Or
43SigneVelasquezAp #868-3039 Eget St.400.002013-03-12 15:20:00Ca
44ZoritaCamachoP.O. Box 651, 3340 Quis Av.400.002013-03-12 15:20:00Or
45GlennaCurtis953-7965 Enim Ave400.002013-03-12 15:20:00Ca
46QuinCortez4898 Ridiculus St.400.002013-03-12 15:20:00Or
47TalonDaltonP.O. Box 408, 7597 Integer Rd.400.002013-03-12 15:20:00Ca
48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.400.002013-03-12 15:20:00Or
49VernonReyesP.O. Box 971, 7009 Vulputate Street400.002013-03-12 15:20:00Ca
50TallulahHeathP.O. Box 865, 3697 Dis Ave500.002013-03-12 15:20:00Or
51CiaranOlson2721 Et St.500.002013-03-12 15:20:00Ca
52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.500.002013-03-12 15:20:00Or
53QuinnRiceAp #647-6627 Tristique Avenue500.002013-03-12 15:20:00Ca
54WyattPickettAp #128-3130 Vel, Rd.500.002013-03-12 15:20:00Or
55EmeraldCopeland857-5119 Turpis Rd.500.002013-03-12 15:20:00Ca
56JonasQuinnAp #441-7183 Ligula. Street500.002013-03-12 15:20:00Or
57WillaBerg6672 Velit Ave500.002013-03-12 15:20:00Ca
58MalikLee998-9208 In Street500.002013-03-12 15:20:00Or
59CallieMedina1620 Dui. Rd.500.002013-03-12 15:20:00Ca
60LukeMasonP.O. Box 143, 2070 Augue Rd.600.002013-03-12 15:20:00Or
61ShafiraEstrada8824 Ante Street600.002014-03-12 15:20:00Ca
62ElizabethRutledge315-6510 Sit St.600.002014-03-12 15:20:00Or
63PandoraLevine357-3596 Nibh. Ave600.002014-03-12 15:20:00Ca
64HilelPrince845-1229 Sociosqu Rd.600.002014-03-12 15:20:00Or
65RinahTorresAp #492-9328 At St.600.002014-03-12 15:20:00Ca
66YaelHobbsP.O. Box 477, 3896 In Street600.002014-03-12 15:20:00Or
67NevadaNashP.O. Box 251, 1914 Tincidunt Road600.002014-03-12 15:20:00Ca
68MarnyHuffP.O. Box 818, 6086 Ultricies St.600.002014-03-12 15:20:00Or
69KimberleyMilesAp #893-3685 In Road600.002014-03-12 15:20:00Ca
70DuncanFullerAp #197-5216 Iaculis Street700.002014-03-12 15:20:00Or
71YardleyLeblancP.O. Box 938, 1278 Sit Ave700.002014-03-12 15:20:00Ca
72HamishBrewerAp #854-781 Quisque St.700.002014-03-12 15:20:00Or
73PetraMoon453-6609 Curabitur Street700.002014-03-12 15:20:00Ca
74ReeseEstradaAp #382-3313 Malesuada St.700.002014-03-12 15:20:00Or
75GageHiggins7443 Eu Street700.002014-03-12 15:20:00Ca
76ZacheryCamachoAp #795-4143 Quam. St.700.002014-03-12 15:20:00Or
77KellyGarnerP.O. Box 895, 2843 Cras Rd.700.002014-03-12 15:20:00Ca
78HanaeCarr9440 Amet St.700.002014-03-12 15:20:00Or
79AnnAlston884-7948 Dictum Road700.002014-03-12 15:20:00Ca
80ChancellorCobbP.O. Box 889, 5978 Ac Avenue800.002014-03-12 15:20:00Or
81DorothyHarrell6974 Tristique Ave800.002010-03-12 15:20:00Ca
82VaughanLeon1610 Luctus Av.800.002010-03-12 15:20:00Or
83WynneJimenez321-9171 Felis. Avenue800.002010-03-12 15:20:00Ca
84WillaMendoza489-182 Sed Av.800.002010-03-12 15:20:00Or
85CamdenGoodwin4579 Ante St.800.002010-03-12 15:20:00Ca
86IfeomaFrenchP.O. Box 160, 8769 Integer Road800.002010-03-12 15:20:00Or
87RamonaStrong1666 Ridiculus Avenue800.002010-03-12 15:20:00Ca
88BrettRamosAp #579-9879 Et, Road800.002010-03-12 15:20:00Or
89UllaGray595-7066 Malesuada Road800.002010-03-12 15:20:00Ca
90KevynMccallP.O. Box 968, 1420 Aenean Avenue900.002010-03-12 15:20:00Or
91GenevieveWilkins908 Turpis. Street900.002010-03-12 15:20:00Ca
92ThaneOneil6766 Lectus St.900.002010-03-12 15:20:00Or
93MarikoClineP.O. Box 329, 5375 Ac St.900.002010-03-12 15:20:00Ca
94LaelMclean500-7010 Sit St.900.002010-03-12 15:20:00Or
95WinifredHopperAp #140-8982 Velit Avenue900.002010-03-12 15:20:00Ca
96RafaelEnglandP.O. Box 405, 7857 Eget Av.900.002010-03-12 15:20:00Or
97DanaCarter814-601 Purus. Av.900.002010-03-12 15:20:00Ca
98JulietBattleAp #535-1965 Cursus St.900.002010-03-12 15:20:00Or
99WynterVincent626-8492 Mollis Avenue900.002010-03-12 15:20:00Ca
100WangMitchell4023 Lacinia. Ave100.002010-03-12 15:20:00Or

View file

@ -0,0 +1,3 @@
1|foo line1|key11:value11,key12:value12,key13:value13|a,b,c|one,two
2|bar line2|key21:value21,key22:value22,key23:value23|d,e,f|three,four
3|baz line3|key31:value31,key32:value32,key33:value33|g,h,i|five,six

View file

@ -0,0 +1,3 @@
1|foo|part1
2|bar|part2
3|baz|part2

View file

@ -0,0 +1,21 @@
100|1|1|1.0|0.0|abc
101|2|2|1.1|0.3|def
102|3|3|1.2|0.6|ghi
103|1|4|1.3|0.9|jkl
104|2|5|1.4|1.2|mno
105|3|1|1.0|1.5|pqr
106|1|2|1.1|1.8|stu
107|2|3|1.2|2.1|vwx
108|3|4|1.3|2.4|yza
109|1|5|1.4|2.7|bcd
110|2|1|1.0|3.0|efg
111|3|2|1.1|3.3|hij
112|1|3|1.2|3.6|klm
113|2|4|1.3|3.9|nop
114|3|5|1.4|4.2|qrs
115|1|1|1.0|4.5|tuv
116|2|2|1.1|4.8|wxy
117|3|3|1.2|5.1|zab
118|1|4|1.3|5.4|cde
119|2|5|1.4|5.7|fgh
120|3|1|1.0|6.0|ijk

View file

@ -1,5 +1,5 @@
John 23
Tom 17
Jim 31
Boby 9
Paul 51
John23
Tom17
Jim31
Boby9
Paul51

View file

@ -0,0 +1,3 @@
Sean29
Tim47
Pooh21

View file

@ -0,0 +1,4 @@
John Doe100000.0Mary SmithTodd JonesFederal Taxes.2State Taxes.05Insurance.11 Michigan Ave.ChicagoIL60600
Mary Smith80000.0Jeremy KingFederal Taxes.2State Taxes. 05Insurance.1100 Ontario St.ChicagoIL60601
Todd Jones70000.0Federal Taxes.15State Taxes.03Insurance. 1200 Chicago Ave.Oak ParkIL60700
Jeremy King60000.0Federal Taxes.15State Taxes.03Insurance. 1300 Obscure Dr.ObscuriaIL60100

View file

@ -0,0 +1 @@
{"a" : "2" ,"b" : "blah"}

View file

@ -1,2 +1,2 @@
../data/files/T1.txt
../data/files/T3.txt
../../data/files/T1.txt
../../data/files/T3.txt

View file

@ -1 +1 @@
../data/files/T2.txt
../../data/files/T2.txt

Binary file not shown.

View file

@ -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.
#
import sys
import re
line = sys.stdin.readline()
x = 1
while line:
tem = sys.stdin.readline()
if line == tem:
x = x + 1
else:
print str(x).strip()+'\t'+re.sub('\t','_',line.strip())
line = tem
x = 1

View file

@ -1,309 +0,0 @@
0.0 0 0 0 3
2.0 0 2 0 1
4.0 0 4 0 1
5.0 0 15 0 3
8.0 0 8 0 1
9.0 0 9 0 1
10.0 0 10 0 1
11.0 0 11 0 1
12.0 0 24 0 2
15.0 0 30 0 2
17.0 0 17 0 1
18.0 0 36 0 2
19.0 0 19 0 1
20.0 0 20 0 1
24.0 0 48 0 2
26.0 0 52 0 2
27.0 0 27 0 1
28.0 0 28 0 1
30.0 0 30 0 1
33.0 0 33 0 1
34.0 0 34 0 1
35.0 0 105 0 3
37.0 0 74 0 2
41.0 0 41 0 1
42.0 0 84 0 2
43.0 0 43 0 1
44.0 0 44 0 1
47.0 0 47 0 1
51.0 0 102 0 2
53.0 0 53 0 1
54.0 0 54 0 1
57.0 0 57 0 1
58.0 0 116 0 2
64.0 0 64 0 1
65.0 0 65 0 1
66.0 0 66 0 1
67.0 0 134 0 2
69.0 0 69 0 1
70.0 0 210 0 3
72.0 0 144 0 2
74.0 0 74 0 1
76.0 0 152 0 2
77.0 0 77 0 1
78.0 0 78 0 1
80.0 0 80 0 1
82.0 0 82 0 1
83.0 0 166 0 2
84.0 0 168 0 2
85.0 0 85 0 1
86.0 0 86 0 1
87.0 0 87 0 1
90.0 0 270 0 3
92.0 0 92 0 1
95.0 0 190 0 2
96.0 0 96 0 1
97.0 0 194 0 2
98.0 0 196 0 2
100.0 0 200 0 2
103.0 0 206 0 2
104.0 0 208 0 2
105.0 0 105 0 1
111.0 0 111 0 1
113.0 0 226 0 2
114.0 0 114 0 1
116.0 0 116 0 1
118.0 0 236 0 2
119.0 0 357 0 3
120.0 0 240 0 2
125.0 0 250 0 2
126.0 0 126 0 1
128.0 0 384 0 3
129.0 0 258 0 2
131.0 0 131 0 1
133.0 0 133 0 1
134.0 0 268 0 2
136.0 0 136 0 1
137.0 0 274 0 2
138.0 0 552 0 4
143.0 0 143 0 1
145.0 0 145 0 1
146.0 0 292 0 2
149.0 0 298 0 2
150.0 0 150 0 1
152.0 0 304 0 2
153.0 0 153 0 1
155.0 0 155 0 1
156.0 0 156 0 1
157.0 0 157 0 1
158.0 0 158 0 1
160.0 0 160 0 1
162.0 0 162 0 1
163.0 0 163 0 1
164.0 0 328 0 2
165.0 0 330 0 2
166.0 0 166 0 1
167.0 0 501 0 3
168.0 0 168 0 1
169.0 0 676 0 4
170.0 0 170 0 1
172.0 0 344 0 2
174.0 0 348 0 2
175.0 0 350 0 2
176.0 0 352 0 2
177.0 0 177 0 1
178.0 0 178 0 1
179.0 0 358 0 2
180.0 0 180 0 1
181.0 0 181 0 1
183.0 0 183 0 1
186.0 0 186 0 1
187.0 0 561 0 3
189.0 0 189 0 1
190.0 0 190 0 1
191.0 0 382 0 2
192.0 0 192 0 1
193.0 0 579 0 3
194.0 0 194 0 1
195.0 0 390 0 2
196.0 0 196 0 1
197.0 0 394 0 2
199.0 0 597 0 3
200.0 0 400 0 2
201.0 0 201 0 1
202.0 0 202 0 1
203.0 0 406 0 2
205.0 0 410 0 2
207.0 0 414 0 2
208.0 0 624 0 3
209.0 0 418 0 2
213.0 0 426 0 2
214.0 0 214 0 1
216.0 0 432 0 2
217.0 0 434 0 2
218.0 0 218 0 1
219.0 0 438 0 2
221.0 0 442 0 2
222.0 0 222 0 1
223.0 0 446 0 2
224.0 0 448 0 2
226.0 0 226 0 1
228.0 0 228 0 1
229.0 0 458 0 2
230.0 0 1150 0 5
233.0 0 466 0 2
235.0 0 235 0 1
237.0 0 474 0 2
238.0 0 476 0 2
239.0 0 478 0 2
241.0 0 241 0 1
242.0 0 484 0 2
244.0 0 244 0 1
247.0 0 247 0 1
248.0 0 248 0 1
249.0 0 249 0 1
252.0 0 252 0 1
255.0 0 510 0 2
256.0 0 512 0 2
257.0 0 257 0 1
258.0 0 258 0 1
260.0 0 260 0 1
262.0 0 262 0 1
263.0 0 263 0 1
265.0 0 530 0 2
266.0 0 266 0 1
272.0 0 544 0 2
273.0 0 819 0 3
274.0 0 274 0 1
275.0 0 275 0 1
277.0 0 1108 0 4
278.0 0 556 0 2
280.0 0 560 0 2
281.0 0 562 0 2
282.0 0 564 0 2
283.0 0 283 0 1
284.0 0 284 0 1
285.0 0 285 0 1
286.0 0 286 0 1
287.0 0 287 0 1
288.0 0 576 0 2
289.0 0 289 0 1
291.0 0 291 0 1
292.0 0 292 0 1
296.0 0 296 0 1
298.0 0 894 0 3
302.0 0 302 0 1
305.0 0 305 0 1
306.0 0 306 0 1
307.0 0 614 0 2
308.0 0 308 0 1
309.0 0 618 0 2
310.0 0 310 0 1
311.0 0 933 0 3
315.0 0 315 0 1
316.0 0 948 0 3
317.0 0 634 0 2
318.0 0 954 0 3
321.0 0 642 0 2
322.0 0 644 0 2
323.0 0 323 0 1
325.0 0 650 0 2
327.0 0 981 0 3
331.0 0 662 0 2
332.0 0 332 0 1
333.0 0 666 0 2
335.0 0 335 0 1
336.0 0 336 0 1
338.0 0 338 0 1
339.0 0 339 0 1
341.0 0 341 0 1
342.0 0 684 0 2
344.0 0 688 0 2
345.0 0 345 0 1
348.0 0 1740 0 5
351.0 0 351 0 1
353.0 0 706 0 2
356.0 0 356 0 1
360.0 0 360 0 1
362.0 0 362 0 1
364.0 0 364 0 1
365.0 0 365 0 1
366.0 0 366 0 1
367.0 0 734 0 2
368.0 0 368 0 1
369.0 0 1107 0 3
373.0 0 373 0 1
374.0 0 374 0 1
375.0 0 375 0 1
377.0 0 377 0 1
378.0 0 378 0 1
379.0 0 379 0 1
382.0 0 764 0 2
384.0 0 1152 0 3
386.0 0 386 0 1
389.0 0 389 0 1
392.0 0 392 0 1
393.0 0 393 0 1
394.0 0 394 0 1
395.0 0 790 0 2
396.0 0 1188 0 3
397.0 0 794 0 2
399.0 0 798 0 2
400.0 0 400 0 1
401.0 0 2005 0 5
402.0 0 402 0 1
403.0 0 1209 0 3
404.0 0 808 0 2
406.0 0 1624 0 4
407.0 0 407 0 1
409.0 0 1227 0 3
411.0 0 411 0 1
413.0 0 826 0 2
414.0 0 828 0 2
417.0 0 1251 0 3
418.0 0 418 0 1
419.0 0 419 0 1
421.0 0 421 0 1
424.0 0 848 0 2
427.0 0 427 0 1
429.0 0 858 0 2
430.0 0 1290 0 3
431.0 0 1293 0 3
432.0 0 432 0 1
435.0 0 435 0 1
436.0 0 436 0 1
437.0 0 437 0 1
438.0 0 1314 0 3
439.0 0 878 0 2
443.0 0 443 0 1
444.0 0 444 0 1
446.0 0 446 0 1
448.0 0 448 0 1
449.0 0 449 0 1
452.0 0 452 0 1
453.0 0 453 0 1
454.0 0 1362 0 3
455.0 0 455 0 1
457.0 0 457 0 1
458.0 0 916 0 2
459.0 0 918 0 2
460.0 0 460 0 1
462.0 0 924 0 2
463.0 0 926 0 2
466.0 0 1398 0 3
467.0 0 467 0 1
468.0 0 1872 0 4
469.0 0 2345 0 5
470.0 0 470 0 1
472.0 0 472 0 1
475.0 0 475 0 1
477.0 0 477 0 1
478.0 0 956 0 2
479.0 0 479 0 1
480.0 0 1440 0 3
481.0 0 481 0 1
482.0 0 482 0 1
483.0 0 483 0 1
484.0 0 484 0 1
485.0 0 485 0 1
487.0 0 487 0 1
489.0 0 1956 0 4
490.0 0 490 0 1
491.0 0 491 0 1
492.0 0 984 0 2
493.0 0 493 0 1
494.0 0 494 0 1
495.0 0 495 0 1
496.0 0 496 0 1
497.0 0 497 0 1
498.0 0 1494 0 3

View file

@ -1,309 +0,0 @@
0.0 0.0 0 0 0 3
0.0 2.0 0 2 0 1
0.0 4.0 0 4 0 1
0.0 5.0 0 15 0 3
0.0 8.0 0 8 0 1
0.0 9.0 0 9 0 1
0.0 10.0 0 10 0 1
0.0 11.0 0 11 0 1
0.0 12.0 0 24 0 2
0.0 15.0 0 30 0 2
0.0 17.0 0 17 0 1
0.0 18.0 0 36 0 2
0.0 19.0 0 19 0 1
0.0 20.0 0 20 0 1
0.0 24.0 0 48 0 2
0.0 26.0 0 52 0 2
0.0 27.0 0 27 0 1
0.0 28.0 0 28 0 1
0.0 30.0 0 30 0 1
0.0 33.0 0 33 0 1
0.0 34.0 0 34 0 1
0.0 35.0 0 105 0 3
0.0 37.0 0 74 0 2
0.0 41.0 0 41 0 1
0.0 42.0 0 84 0 2
0.0 43.0 0 43 0 1
0.0 44.0 0 44 0 1
0.0 47.0 0 47 0 1
0.0 51.0 0 102 0 2
0.0 53.0 0 53 0 1
0.0 54.0 0 54 0 1
0.0 57.0 0 57 0 1
0.0 58.0 0 116 0 2
0.0 64.0 0 64 0 1
0.0 65.0 0 65 0 1
0.0 66.0 0 66 0 1
0.0 67.0 0 134 0 2
0.0 69.0 0 69 0 1
0.0 70.0 0 210 0 3
0.0 72.0 0 144 0 2
0.0 74.0 0 74 0 1
0.0 76.0 0 152 0 2
0.0 77.0 0 77 0 1
0.0 78.0 0 78 0 1
0.0 80.0 0 80 0 1
0.0 82.0 0 82 0 1
0.0 83.0 0 166 0 2
0.0 84.0 0 168 0 2
0.0 85.0 0 85 0 1
0.0 86.0 0 86 0 1
0.0 87.0 0 87 0 1
0.0 90.0 0 270 0 3
0.0 92.0 0 92 0 1
0.0 95.0 0 190 0 2
0.0 96.0 0 96 0 1
0.0 97.0 0 194 0 2
0.0 98.0 0 196 0 2
0.0 100.0 0 200 0 2
0.0 103.0 0 206 0 2
0.0 104.0 0 208 0 2
0.0 105.0 0 105 0 1
0.0 111.0 0 111 0 1
0.0 113.0 0 226 0 2
0.0 114.0 0 114 0 1
0.0 116.0 0 116 0 1
0.0 118.0 0 236 0 2
0.0 119.0 0 357 0 3
0.0 120.0 0 240 0 2
0.0 125.0 0 250 0 2
0.0 126.0 0 126 0 1
0.0 128.0 0 384 0 3
0.0 129.0 0 258 0 2
0.0 131.0 0 131 0 1
0.0 133.0 0 133 0 1
0.0 134.0 0 268 0 2
0.0 136.0 0 136 0 1
0.0 137.0 0 274 0 2
0.0 138.0 0 552 0 4
0.0 143.0 0 143 0 1
0.0 145.0 0 145 0 1
0.0 146.0 0 292 0 2
0.0 149.0 0 298 0 2
0.0 150.0 0 150 0 1
0.0 152.0 0 304 0 2
0.0 153.0 0 153 0 1
0.0 155.0 0 155 0 1
0.0 156.0 0 156 0 1
0.0 157.0 0 157 0 1
0.0 158.0 0 158 0 1
0.0 160.0 0 160 0 1
0.0 162.0 0 162 0 1
0.0 163.0 0 163 0 1
0.0 164.0 0 328 0 2
0.0 165.0 0 330 0 2
0.0 166.0 0 166 0 1
0.0 167.0 0 501 0 3
0.0 168.0 0 168 0 1
0.0 169.0 0 676 0 4
0.0 170.0 0 170 0 1
0.0 172.0 0 344 0 2
0.0 174.0 0 348 0 2
0.0 175.0 0 350 0 2
0.0 176.0 0 352 0 2
0.0 177.0 0 177 0 1
0.0 178.0 0 178 0 1
0.0 179.0 0 358 0 2
0.0 180.0 0 180 0 1
0.0 181.0 0 181 0 1
0.0 183.0 0 183 0 1
0.0 186.0 0 186 0 1
0.0 187.0 0 561 0 3
0.0 189.0 0 189 0 1
0.0 190.0 0 190 0 1
0.0 191.0 0 382 0 2
0.0 192.0 0 192 0 1
0.0 193.0 0 579 0 3
0.0 194.0 0 194 0 1
0.0 195.0 0 390 0 2
0.0 196.0 0 196 0 1
0.0 197.0 0 394 0 2
0.0 199.0 0 597 0 3
0.0 200.0 0 400 0 2
0.0 201.0 0 201 0 1
0.0 202.0 0 202 0 1
0.0 203.0 0 406 0 2
0.0 205.0 0 410 0 2
0.0 207.0 0 414 0 2
0.0 208.0 0 624 0 3
0.0 209.0 0 418 0 2
0.0 213.0 0 426 0 2
0.0 214.0 0 214 0 1
0.0 216.0 0 432 0 2
0.0 217.0 0 434 0 2
0.0 218.0 0 218 0 1
0.0 219.0 0 438 0 2
0.0 221.0 0 442 0 2
0.0 222.0 0 222 0 1
0.0 223.0 0 446 0 2
0.0 224.0 0 448 0 2
0.0 226.0 0 226 0 1
0.0 228.0 0 228 0 1
0.0 229.0 0 458 0 2
0.0 230.0 0 1150 0 5
0.0 233.0 0 466 0 2
0.0 235.0 0 235 0 1
0.0 237.0 0 474 0 2
0.0 238.0 0 476 0 2
0.0 239.0 0 478 0 2
0.0 241.0 0 241 0 1
0.0 242.0 0 484 0 2
0.0 244.0 0 244 0 1
0.0 247.0 0 247 0 1
0.0 248.0 0 248 0 1
0.0 249.0 0 249 0 1
0.0 252.0 0 252 0 1
0.0 255.0 0 510 0 2
0.0 256.0 0 512 0 2
0.0 257.0 0 257 0 1
0.0 258.0 0 258 0 1
0.0 260.0 0 260 0 1
0.0 262.0 0 262 0 1
0.0 263.0 0 263 0 1
0.0 265.0 0 530 0 2
0.0 266.0 0 266 0 1
0.0 272.0 0 544 0 2
0.0 273.0 0 819 0 3
0.0 274.0 0 274 0 1
0.0 275.0 0 275 0 1
0.0 277.0 0 1108 0 4
0.0 278.0 0 556 0 2
0.0 280.0 0 560 0 2
0.0 281.0 0 562 0 2
0.0 282.0 0 564 0 2
0.0 283.0 0 283 0 1
0.0 284.0 0 284 0 1
0.0 285.0 0 285 0 1
0.0 286.0 0 286 0 1
0.0 287.0 0 287 0 1
0.0 288.0 0 576 0 2
0.0 289.0 0 289 0 1
0.0 291.0 0 291 0 1
0.0 292.0 0 292 0 1
0.0 296.0 0 296 0 1
0.0 298.0 0 894 0 3
0.0 302.0 0 302 0 1
0.0 305.0 0 305 0 1
0.0 306.0 0 306 0 1
0.0 307.0 0 614 0 2
0.0 308.0 0 308 0 1
0.0 309.0 0 618 0 2
0.0 310.0 0 310 0 1
0.0 311.0 0 933 0 3
0.0 315.0 0 315 0 1
0.0 316.0 0 948 0 3
0.0 317.0 0 634 0 2
0.0 318.0 0 954 0 3
0.0 321.0 0 642 0 2
0.0 322.0 0 644 0 2
0.0 323.0 0 323 0 1
0.0 325.0 0 650 0 2
0.0 327.0 0 981 0 3
0.0 331.0 0 662 0 2
0.0 332.0 0 332 0 1
0.0 333.0 0 666 0 2
0.0 335.0 0 335 0 1
0.0 336.0 0 336 0 1
0.0 338.0 0 338 0 1
0.0 339.0 0 339 0 1
0.0 341.0 0 341 0 1
0.0 342.0 0 684 0 2
0.0 344.0 0 688 0 2
0.0 345.0 0 345 0 1
0.0 348.0 0 1740 0 5
0.0 351.0 0 351 0 1
0.0 353.0 0 706 0 2
0.0 356.0 0 356 0 1
0.0 360.0 0 360 0 1
0.0 362.0 0 362 0 1
0.0 364.0 0 364 0 1
0.0 365.0 0 365 0 1
0.0 366.0 0 366 0 1
0.0 367.0 0 734 0 2
0.0 368.0 0 368 0 1
0.0 369.0 0 1107 0 3
0.0 373.0 0 373 0 1
0.0 374.0 0 374 0 1
0.0 375.0 0 375 0 1
0.0 377.0 0 377 0 1
0.0 378.0 0 378 0 1
0.0 379.0 0 379 0 1
0.0 382.0 0 764 0 2
0.0 384.0 0 1152 0 3
0.0 386.0 0 386 0 1
0.0 389.0 0 389 0 1
0.0 392.0 0 392 0 1
0.0 393.0 0 393 0 1
0.0 394.0 0 394 0 1
0.0 395.0 0 790 0 2
0.0 396.0 0 1188 0 3
0.0 397.0 0 794 0 2
0.0 399.0 0 798 0 2
0.0 400.0 0 400 0 1
0.0 401.0 0 2005 0 5
0.0 402.0 0 402 0 1
0.0 403.0 0 1209 0 3
0.0 404.0 0 808 0 2
0.0 406.0 0 1624 0 4
0.0 407.0 0 407 0 1
0.0 409.0 0 1227 0 3
0.0 411.0 0 411 0 1
0.0 413.0 0 826 0 2
0.0 414.0 0 828 0 2
0.0 417.0 0 1251 0 3
0.0 418.0 0 418 0 1
0.0 419.0 0 419 0 1
0.0 421.0 0 421 0 1
0.0 424.0 0 848 0 2
0.0 427.0 0 427 0 1
0.0 429.0 0 858 0 2
0.0 430.0 0 1290 0 3
0.0 431.0 0 1293 0 3
0.0 432.0 0 432 0 1
0.0 435.0 0 435 0 1
0.0 436.0 0 436 0 1
0.0 437.0 0 437 0 1
0.0 438.0 0 1314 0 3
0.0 439.0 0 878 0 2
0.0 443.0 0 443 0 1
0.0 444.0 0 444 0 1
0.0 446.0 0 446 0 1
0.0 448.0 0 448 0 1
0.0 449.0 0 449 0 1
0.0 452.0 0 452 0 1
0.0 453.0 0 453 0 1
0.0 454.0 0 1362 0 3
0.0 455.0 0 455 0 1
0.0 457.0 0 457 0 1
0.0 458.0 0 916 0 2
0.0 459.0 0 918 0 2
0.0 460.0 0 460 0 1
0.0 462.0 0 924 0 2
0.0 463.0 0 926 0 2
0.0 466.0 0 1398 0 3
0.0 467.0 0 467 0 1
0.0 468.0 0 1872 0 4
0.0 469.0 0 2345 0 5
0.0 470.0 0 470 0 1
0.0 472.0 0 472 0 1
0.0 475.0 0 475 0 1
0.0 477.0 0 477 0 1
0.0 478.0 0 956 0 2
0.0 479.0 0 479 0 1
0.0 480.0 0 1440 0 3
0.0 481.0 0 481 0 1
0.0 482.0 0 482 0 1
0.0 483.0 0 483 0 1
0.0 484.0 0 484 0 1
0.0 485.0 0 485 0 1
0.0 487.0 0 487 0 1
0.0 489.0 0 1956 0 4
0.0 490.0 0 490 0 1
0.0 491.0 0 491 0 1
0.0 492.0 0 984 0 2
0.0 493.0 0 493 0 1
0.0 494.0 0 494 0 1
0.0 495.0 0 495 0 1
0.0 496.0 0 496 0 1
0.0 497.0 0 497 0 1
0.0 498.0 0 1494 0 3

View file

@ -1,309 +0,0 @@
0 3
0 1
0 1
0 3
0 1
0 1
0 1
0 1
0 2
0 2
0 1
0 2
0 1
0 1
0 2
0 2
0 1
0 1
0 1
0 1
0 1
0 3
0 2
0 1
0 2
0 1
0 1
0 1
0 2
0 1
0 1
0 1
0 2
0 1
0 1
0 1
0 2
0 1
0 3
0 2
0 1
0 2
0 1
0 1
0 1
0 1
0 2
0 2
0 1
0 1
0 1
0 3
0 1
0 2
0 1
0 2
0 2
0 2
0 2
0 2
0 1
0 1
0 2
0 1
0 1
0 2
0 3
0 2
0 2
0 1
0 3
0 2
0 1
0 1
0 2
0 1
0 2
0 4
0 1
0 1
0 2
0 2
0 1
0 2
0 1
0 1
0 1
0 1
0 1
0 1
0 1
0 1
0 2
0 2
0 1
0 3
0 1
0 4
0 1
0 2
0 2
0 2
0 2
0 1
0 1
0 2
0 1
0 1
0 1
0 1
0 3
0 1
0 1
0 2
0 1
0 3
0 1
0 2
0 1
0 2
0 3
0 2
0 1
0 1
0 2
0 2
0 2
0 3
0 2
0 2
0 1
0 2
0 2
0 1
0 2
0 2
0 1
0 2
0 2
0 1
0 1
0 2
0 5
0 2
0 1
0 2
0 2
0 2
0 1
0 2
0 1
0 1
0 1
0 1
0 1
0 2
0 2
0 1
0 1
0 1
0 1
0 1
0 2
0 1
0 2
0 3
0 1
0 1
0 4
0 2
0 2
0 2
0 2
0 1
0 1
0 1
0 1
0 1
0 2
0 1
0 1
0 1
0 1
0 3
0 1
0 1
0 1
0 2
0 1
0 2
0 1
0 3
0 1
0 3
0 2
0 3
0 2
0 2
0 1
0 2
0 3
0 2
0 1
0 2
0 1
0 1
0 1
0 1
0 1
0 2
0 2
0 1
0 5
0 1
0 2
0 1
0 1
0 1
0 1
0 1
0 1
0 2
0 1
0 3
0 1
0 1
0 1
0 1
0 1
0 1
0 2
0 3
0 1
0 1
0 1
0 1
0 1
0 2
0 3
0 2
0 2
0 1
0 5
0 1
0 3
0 2
0 4
0 1
0 3
0 1
0 2
0 2
0 3
0 1
0 1
0 1
0 2
0 1
0 2
0 3
0 3
0 1
0 1
0 1
0 1
0 3
0 2
0 1
0 1
0 1
0 1
0 1
0 1
0 1
0 3
0 1
0 1
0 2
0 2
0 1
0 2
0 2
0 3
0 1
0 4
0 5
0 1
0 1
0 1
0 1
0 2
0 1
0 3
0 1
0 1
0 1
0 1
0 1
0 1
0 4
0 1
0 1
0 2
0 1
0 1
0 1
0 1
0 1
0 3

View file

@ -172,4 +172,4 @@
194 val_194
414 val_414
126 val_126
169 val_169
169 val_169

View file

@ -306,4 +306,4 @@
96.0 val_95
97.0 val_96
98.0 val_97
99.0 val_98
99.0 val_98

Some files were not shown because too many files have changed in this diff Show more