You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by br...@apache.org on 2014/08/13 00:25:28 UTC

svn commit: r1617610 - in /hive/trunk: ant/ ant/src/org/apache/hadoop/hive/ant/ itests/qtest/ itests/src/ itests/src/test/ itests/src/test/resources/

Author: brock
Date: Tue Aug 12 22:25:28 2014
New Revision: 1617610

URL: http://svn.apache.org/r1617610
Log:
HIVE-7691 - Improve maintainability of testconfiguration.properties (Reviewed by Szehon via Brock)

Added:
    hive/trunk/itests/src/
    hive/trunk/itests/src/test/
    hive/trunk/itests/src/test/resources/
    hive/trunk/itests/src/test/resources/testconfiguration.properties
Removed:
    hive/trunk/itests/qtest/testconfiguration.properties
Modified:
    hive/trunk/ant/pom.xml
    hive/trunk/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
    hive/trunk/itests/qtest/pom.xml

Modified: hive/trunk/ant/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/ant/pom.xml?rev=1617610&r1=1617609&r2=1617610&view=diff
==============================================================================
--- hive/trunk/ant/pom.xml (original)
+++ hive/trunk/ant/pom.xml Tue Aug 12 22:25:28 2014
@@ -39,6 +39,11 @@
       <artifactId>commons-lang</artifactId>
       <version>${commons-lang.version}</version>
     </dependency>
+      <dependency>
+        <groupId>com.google.guava</groupId>
+        <artifactId>guava</artifactId>
+        <version>${guava.version}</version>
+      </dependency>
     <dependency>
       <groupId>org.apache.ant</groupId>
       <artifactId>ant</artifactId>

Modified: hive/trunk/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java?rev=1617610&r1=1617609&r2=1617610&view=diff
==============================================================================
--- hive/trunk/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java (original)
+++ hive/trunk/ant/src/org/apache/hadoop/hive/ant/QTestGenTask.java Tue Aug 12 22:25:28 2014
@@ -32,10 +32,11 @@ import java.util.ArrayList;
 import java.util.regex.Pattern;
 import java.util.HashMap;
 
+import com.google.common.base.Splitter;
+import com.google.common.collect.Sets;
 import org.apache.commons.lang.StringUtils;
 import org.apache.tools.ant.BuildException;
 import org.apache.tools.ant.Task;
-
 import org.apache.velocity.app.VelocityEngine;
 import org.apache.velocity.Template;
 import org.apache.velocity.VelocityContext;
@@ -45,6 +46,9 @@ import org.apache.velocity.exception.Res
 import org.apache.velocity.runtime.RuntimeConstants;
 
 public class QTestGenTask extends Task {
+   private static final Splitter CSV_SPLITTER = Splitter.on(',')
+       .trimResults()
+       .omitEmptyStrings();
 
   public class IncludeFilter implements FileFilter {
 
@@ -74,9 +78,8 @@ public class QTestGenTask extends Task {
       }
       return true;
     }
-    
   }
-  
+
   public class DisabledQFileFilter extends IncludeFilter {
     public DisabledQFileFilter(Set<String> includeOnly) {
       super(includeOnly);
@@ -87,17 +90,16 @@ public class QTestGenTask extends Task {
         return false;
       }
       return !fpath.isDirectory() && fpath.getName().endsWith(".q.disabled");
-    }  
+    }
   }
-  
+
   public class QFileRegexFilter extends QFileFilter {
     Pattern filterPattern;
-    
     public QFileRegexFilter(String filter, Set<String> includeOnly) {
       super(includeOnly);
       filterPattern = Pattern.compile(filter);
     }
-    
+
     public boolean accept(File filePath) {
       if (!super.accept(filePath)) {
         return false;
@@ -110,17 +112,17 @@ public class QTestGenTask extends Task {
   private List<String> templatePaths = new ArrayList<String>();
 
   private String hiveRootDirectory;
-  
+
   private String outputDirectory;
- 
+
   private String queryDirectory;
- 
+
   private String queryFile;
 
   private String includeQueryFile;
 
   private String excludeQueryFile;
-  
+
   private String queryFileRegex;
 
   private String resultsDirectory;
@@ -138,7 +140,7 @@ public class QTestGenTask extends Task {
   private String hiveConfDir;
 
   private String runDisabled;
-  
+
   private String hadoopVersion;
 
   private String initScript;
@@ -160,7 +162,7 @@ public class QTestGenTask extends Task {
   public String getHiveConfDir() {
     return hiveConfDir;
   }
-  
+
   public void setClusterMode(String clusterMode) {
     this.clusterMode = clusterMode;
   }
@@ -228,10 +230,10 @@ public class QTestGenTask extends Task {
   public String getHiveRootDirectory() {
     return hiveRootDirectory;
   }
-  
+
   public void setTemplatePath(String templatePath) throws Exception {
     templatePaths.clear();
-    for (String relativePath : templatePath.split(",")) {
+    for (String relativePath : CSV_SPLITTER.split(templatePath)) {
       templatePaths.add(project.resolveFile(relativePath).getCanonicalPath());
     }
     System.out.println("Template Path:" + getTemplatePath());
@@ -336,7 +338,7 @@ public class QTestGenTask extends Task {
 
     Set<String> includeOnly = null;
     if (includeQueryFile != null && !includeQueryFile.isEmpty()) {
-      includeOnly = new HashSet<String>(Arrays.asList(includeQueryFile.split(",")));
+      includeOnly = Sets.<String>newHashSet(CSV_SPLITTER.split(includeQueryFile));
     }
 
     List<File> qFiles;
@@ -346,7 +348,7 @@ public class QTestGenTask extends Task {
     File outDir = null;
     File resultsDir = null;
     File logDir = null;
-    
+
     try {
       // queryDirectory should not be null
       queryDir = new File(queryDirectory);
@@ -355,7 +357,7 @@ public class QTestGenTask extends Task {
       Set<File> testFiles = new HashSet<File>();
       if (queryFile != null && !queryFile.equals("")) {
         // The user may have passed a list of files - comma separated
-        for (String qFile : queryFile.split(",")) {
+        for (String qFile : CSV_SPLITTER.split(queryFile)) {
           if (includeOnly != null && !includeOnly.contains(qFile)) {
             continue;
           }
@@ -366,7 +368,7 @@ public class QTestGenTask extends Task {
           }
         }
       } else if (queryFileRegex != null && !queryFileRegex.equals("")) {
-        for (String regex : queryFileRegex.split(",")) {
+        for (String regex : CSV_SPLITTER.split(queryFileRegex)) {
           testFiles.addAll(Arrays.asList(queryDir.listFiles(
               new QFileRegexFilter(regex, includeOnly))));
         }
@@ -378,7 +380,7 @@ public class QTestGenTask extends Task {
 
       if (excludeQueryFile != null && !excludeQueryFile.equals("")) {
         // Exclude specified query files, comma separated
-        for (String qFile : excludeQueryFile.split(",")) {
+        for (String qFile : CSV_SPLITTER.split(excludeQueryFile)) {
           if (null != queryDir) {
             testFiles.remove(new File(queryDir, qFile));
           } else {
@@ -488,9 +490,11 @@ public class QTestGenTask extends Task {
       throw new BuildException("Generation failed", e);
     }
   }
+
   private String relativePath(File hiveRootDir, File file) {
     return escapePath(hiveRootDir.toURI().relativize(file.toURI()).getPath());
-  }  
+  }
+
   private static String escapePath(String path) {
     if (System.getProperty("os.name").toLowerCase().startsWith("win")) {
       // Escape the backward slash in CanonicalPath if the unit test runs on windows

Modified: hive/trunk/itests/qtest/pom.xml
URL: http://svn.apache.org/viewvc/hive/trunk/itests/qtest/pom.xml?rev=1617610&r1=1617609&r2=1617610&view=diff
==============================================================================
--- hive/trunk/itests/qtest/pom.xml (original)
+++ hive/trunk/itests/qtest/pom.xml Tue Aug 12 22:25:28 2014
@@ -373,7 +373,7 @@
             </goals>
             <configuration>
               <files>
-                <file>${basedir}/testconfiguration.properties</file>
+                <file>${basedir}/../src/test/resources/testconfiguration.properties</file>
               </files>
             </configuration>
           </execution>

Added: hive/trunk/itests/src/test/resources/testconfiguration.properties
URL: http://svn.apache.org/viewvc/hive/trunk/itests/src/test/resources/testconfiguration.properties?rev=1617610&view=auto
==============================================================================
--- hive/trunk/itests/src/test/resources/testconfiguration.properties (added)
+++ hive/trunk/itests/src/test/resources/testconfiguration.properties Tue Aug 12 22:25:28 2014
@@ -0,0 +1,311 @@
+# NOTE: files should be listed in alphabetical order
+minimr.query.files=auto_sortmerge_join_16.q,\
+  bucket4.q,\
+  bucket5.q,\
+  bucket6.q,\
+  bucket_num_reducers.q,\
+  bucket_num_reducers2.q,\
+  bucketizedhiveinputformat.q,\
+  bucketmapjoin6.q,\
+  bucketmapjoin7.q,\
+  disable_merge_for_bucketing.q,\
+  empty_dir_in_table.q,\
+  external_table_with_space_in_location_path.q,\
+  file_with_header_footer.q,\
+  groupby2.q,\
+  import_exported_table.q,\
+  index_bitmap3.q,\
+  index_bitmap_auto.q,\
+  infer_bucket_sort_bucketed_table.q,\
+  infer_bucket_sort_dyn_part.q,\
+  infer_bucket_sort_map_operators.q,\
+  infer_bucket_sort_merge.q,\
+  infer_bucket_sort_num_buckets.q,\
+  infer_bucket_sort_reducers_power_two.q,\
+  input16_cc.q,\
+  join1.q,\
+  leftsemijoin_mr.q,\
+  list_bucket_dml_10.q,\
+  load_fs2.q,\
+  load_hdfs_file_with_space_in_the_name.q,\
+  optrstat_groupby.q,\
+  parallel_orderby.q,\
+  ql_rewrite_gbtoidx.q,\
+  quotedid_smb.q,\
+  reduce_deduplicate.q,\
+  remote_script.q,\
+  root_dir_external_table.q,\
+  schemeAuthority.q,\
+  schemeAuthority2.q,\
+  scriptfile1.q,\
+  scriptfile1_win.q,\
+  smb_mapjoin_8.q,\
+  stats_counter.q,\
+  stats_counter_partitioned.q,\
+  temp_table_external.q,\
+  truncate_column_buckets.q,\
+  udf_using.q
+
+minitez.query.files.shared=alter_merge_2_orc.q,\
+  alter_merge_orc.q,\
+  alter_merge_stats_orc.q,\
+  auto_join0.q,\
+  auto_join1.q,\
+  bucket2.q,\
+  bucket3.q,\
+  bucket4.q,\
+  count.q,\
+  create_merge_compressed.q,\
+  cross_join.q,\
+  cross_product_check_1.q,\
+  cross_product_check_2.q,\
+  ctas.q,\
+  custom_input_output_format.q,\
+  disable_merge_for_bucketing.q,\
+  dynpart_sort_opt_vectorization.q,\
+  dynpart_sort_optimization.q,\
+  enforce_order.q,\
+  filter_join_breaktask.q,\
+  filter_join_breaktask2.q,\
+  groupby1.q,\
+  groupby2.q,\
+  groupby3.q,\
+  having.q,\
+  insert1.q,\
+  insert_into1.q,\
+  insert_into2.q,\
+  join0.q,\
+  join1.q,\
+  leftsemijoin.q,\
+  limit_pushdown.q,\
+  load_dyn_part1.q,\
+  load_dyn_part2.q,\
+  load_dyn_part3.q,\
+  mapjoin_mapjoin.q,\
+  mapreduce1.q,\
+  mapreduce2.q,\
+  merge1.q,\
+  merge2.q,\
+  metadata_only_queries.q,\
+  metadataonly1.q,\
+  optimize_nullscan.q,\
+  orc_analyze.q,\
+  orc_merge1.q,\
+  orc_merge2.q,\
+  orc_merge3.q,\
+  orc_merge4.q,\
+  ptf.q,\
+  sample1.q,\
+  script_env_var1.q,\
+  script_env_var2.q,\
+  script_pipe.q,\
+  scriptfile1.q,\
+  stats_counter.q,\
+  stats_counter_partitioned.q,\
+  stats_noscan_1.q,\
+  subquery_exists.q,\
+  subquery_in.q,\
+  temp_table.q,\
+  transform1.q,\
+  transform2.q,\
+  transform_ppr1.q,\
+  transform_ppr2.q,\
+  union2.q,\
+  union3.q,\
+  union4.q,\
+  union5.q,\
+  union6.q,\
+  union7.q,\
+  union8.q,\
+  union9.q,\
+  vector_cast_constant.q,\
+  vector_data_types.q,\
+  vector_decimal_aggregate.q,\
+  vector_left_outer_join.q,\
+  vector_string_concat.q,\
+  vectorization_12.q,\
+  vectorization_13.q,\
+  vectorization_14.q,\
+  vectorization_15.q,\
+  vectorization_9.q,\
+  vectorization_part_project.q,\
+  vectorization_short_regress.q,\
+  vectorized_mapjoin.q,\
+  vectorized_nested_mapjoin.q,\
+  vectorized_ptf.q,\
+  vectorized_shufflejoin.q,\
+  vectorized_timestamp_funcs.q
+
+minitez.query.files=bucket_map_join_tez1.q,\
+  bucket_map_join_tez2.q,\
+  mapjoin_decimal.q,\
+  mrr.q,\
+  tez_dml.q,\
+  tez_fsstat.q,\
+  tez_insert_overwrite_local_directory_1.q,\
+  tez_join_hash.q,\
+  tez_join_tests.q,\
+  tez_joins_explain.q,\
+  tez_schema_evolution.q,\
+  tez_union.q
+
+beeline.positive.exclude=add_part_exist.q,\
+  alter1.q,\
+  alter2.q,\
+  alter4.q,\
+  alter5.q,\
+  alter_rename_partition.q,\
+  alter_rename_partition_authorization.q,\
+  archive.q,\
+  archive_corrupt.q,\
+  archive_mr_1806.q,\
+  archive_multi.q,\
+  archive_multi_mr_1806.q,\
+  authorization_1.q,\
+  authorization_2.q,\
+  authorization_4.q,\
+  authorization_5.q,\
+  authorization_6.q,\
+  authorization_7.q,\
+  ba_table1.q,\
+  ba_table2.q,\
+  ba_table3.q,\
+  ba_table_udfs.q,\
+  binary_table_bincolserde.q,\
+  binary_table_colserde.q,\
+  cluster.q,\
+  columnarserde_create_shortcut.q,\
+  combine2.q,\
+  constant_prop.q,\
+  create_nested_type.q,\
+  create_or_replace_view.q,\
+  create_struct_table.q,\
+  create_union_table.q,\
+  database.q,\
+  database_location.q,\
+  database_properties.q,\
+  ddltime.q,\
+  describe_database_json.q,\
+  drop_database_removes_partition_dirs.q,\
+  escape1.q,\
+  escape2.q,\
+  exim_00_nonpart_empty.q,\
+  exim_01_nonpart.q,\
+  exim_02_00_part_empty.q,\
+  exim_02_part.q,\
+  exim_03_nonpart_over_compat.q,\
+  exim_04_all_part.q,\
+  exim_04_evolved_parts.q,\
+  exim_05_some_part.q,\
+  exim_06_one_part.q,\
+  exim_07_all_part_over_nonoverlap.q,\
+  exim_08_nonpart_rename.q,\
+  exim_09_part_spec_nonoverlap.q,\
+  exim_10_external_managed.q,\
+  exim_11_managed_external.q,\
+  exim_12_external_location.q,\
+  exim_13_managed_location.q,\
+  exim_14_managed_location_over_existing.q,\
+  exim_15_external_part.q,\
+  exim_16_part_external.q,\
+  exim_17_part_managed.q,\
+  exim_18_part_external.q,\
+  exim_19_00_part_external_location.q,\
+  exim_19_part_external_location.q,\
+  exim_20_part_managed_location.q,\
+  exim_21_export_authsuccess.q,\
+  exim_22_import_exist_authsuccess.q,\
+  exim_23_import_part_authsuccess.q,\
+  exim_24_import_nonexist_authsuccess.q,\
+  global_limit.q,\
+  groupby_complex_types.q,\
+  groupby_complex_types_multi_single_reducer.q,\
+  index_auth.q,\
+  index_auto.q,\
+  index_auto_empty.q,\
+  index_bitmap.q,\
+  index_bitmap1.q,\
+  index_bitmap2.q,\
+  index_bitmap3.q,\
+  index_bitmap_auto.q,\
+  index_bitmap_rc.q,\
+  index_compact.q,\
+  index_compact_1.q,\
+  index_compact_2.q,\
+  index_compact_3.q,\
+  index_stale_partitioned.q,\
+  init_file.q,\
+  input16.q,\
+  input16_cc.q,\
+  input46.q,\
+  input_columnarserde.q,\
+  input_dynamicserde.q,\
+  input_lazyserde.q,\
+  input_testxpath3.q,\
+  input_testxpath4.q,\
+  insert2_overwrite_partitions.q,\
+  insertexternal1.q,\
+  join_thrift.q,\
+  lateral_view.q,\
+  load_binary_data.q,\
+  load_exist_part_authsuccess.q,\
+  load_nonpart_authsuccess.q,\
+  load_part_authsuccess.q,\
+  loadpart_err.q,\
+  lock1.q,\
+  lock2.q,\
+  lock3.q,\
+  lock4.q,\
+  merge_dynamic_partition.q,\
+  multi_insert.q,\
+  multi_insert_move_tasks_share_dependencies.q,\
+  null_column.q,\
+  ppd_clusterby.q,\
+  query_with_semi.q,\
+  rename_column.q,\
+  sample6.q,\
+  sample_islocalmode_hook.q,\
+  set_processor_namespaces.q,\
+  show_tables.q,\
+  source.q,\
+  split_sample.q,\
+  str_to_map.q,\
+  transform1.q,\
+  udaf_collect_set.q,\
+  udaf_context_ngrams.q,\
+  udaf_histogram_numeric.q,\
+  udaf_ngrams.q,\
+  udaf_percentile_approx.q,\
+  udf_array.q,\
+  udf_bitmap_and.q,\
+  udf_bitmap_or.q,\
+  udf_explode.q,\
+  udf_format_number.q,\
+  udf_map.q,\
+  udf_map_keys.q,\
+  udf_map_values.q,\
+  udf_max.q,\
+  udf_min.q,\
+  udf_named_struct.q,\
+  udf_percentile.q,\
+  udf_printf.q,\
+  udf_sentences.q,\
+  udf_sort_array.q,\
+  udf_split.q,\
+  udf_struct.q,\
+  udf_substr.q,\
+  udf_translate.q,\
+  udf_union.q,\
+  udf_xpath.q,\
+  udtf_stack.q,\
+  view.q,\
+  virtual_column.q
+
+minimr.query.negative.files=cluster_tasklog_retrieval.q,\
+  file_with_header_footer_negative.q,\
+  mapreduce_stack_trace.q,\
+  mapreduce_stack_trace_hadoop20.q,\
+  mapreduce_stack_trace_turnoff.q,\
+  mapreduce_stack_trace_turnoff_hadoop20.q,\
+  minimr_broken_pipe.q,\
+  udf_local_resource.q