/
HiveConf.java
6998 lines (6560 loc) · 430 KB
/
HiveConf.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.conf;
import com.google.common.base.Joiner;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.ZooKeeperHiveHelper;
import org.apache.hadoop.hive.common.classification.InterfaceAudience;
import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.hive.common.type.TimestampTZUtil;
import org.apache.hadoop.hive.conf.Validator.PatternSet;
import org.apache.hadoop.hive.conf.Validator.RangeValidator;
import org.apache.hadoop.hive.conf.Validator.RatioValidator;
import org.apache.hadoop.hive.conf.Validator.SizeValidator;
import org.apache.hadoop.hive.conf.Validator.StringSet;
import org.apache.hadoop.hive.conf.Validator.TimeValidator;
import org.apache.hadoop.hive.conf.Validator.WritableDirectoryValidator;
import org.apache.hadoop.hive.shims.ShimLoader;
import org.apache.hadoop.hive.shims.Utils;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hive.common.HiveCompat;
import org.apache.hive.common.util.SuppressFBWarnings;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.security.auth.login.LoginException;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
import java.net.URI;
import java.net.URL;
import java.net.URLDecoder;
import java.net.URLEncoder;
import java.nio.charset.StandardCharsets;
import java.time.ZoneId;
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* Hive Configuration.
*/
public class HiveConf extends Configuration {
protected String hiveJar;
protected Properties origProp;
protected String auxJars;
private static final Logger LOG = LoggerFactory.getLogger(HiveConf.class);
private static boolean loadMetastoreConfig = false;
private static boolean loadHiveServer2Config = false;
private static URL hiveDefaultURL = null;
private static URL hiveSiteURL = null;
private static URL hivemetastoreSiteUrl = null;
private static URL hiveServer2SiteUrl = null;
private static byte[] confVarByteArray = null;
private static final Map<String, ConfVars> vars = new HashMap<String, ConfVars>();
private static final Map<String, ConfVars> metaConfs = new HashMap<String, ConfVars>();
private final List<String> restrictList = new ArrayList<String>();
private final Set<String> hiddenSet = new HashSet<String>();
private final List<String> rscList = new ArrayList<>();
private Pattern modWhiteListPattern = null;
private static final int LOG_PREFIX_LENGTH = 64;
public enum ResultFileFormat {
INVALID_FORMAT {
@Override
public String toString() {
return "invalid result file format";
}
},
TEXTFILE {
@Override
public String toString() {
return "TextFile";
}
},
SEQUENCEFILE {
@Override
public String toString() {
return "SequenceFile";
}
},
RCFILE {
@Override
public String toString() {
return "RCfile";
}
},
LLAP {
@Override
public String toString() {
return "Llap";
}
};
public static ResultFileFormat getInvalid() {
return INVALID_FORMAT;
}
public static EnumSet<ResultFileFormat> getValidSet() {
return EnumSet.complementOf(EnumSet.of(getInvalid()));
}
public static ResultFileFormat from(String value) {
try {
return valueOf(value.toUpperCase());
} catch (Exception e) {
return getInvalid();
}
}
}
public ResultFileFormat getResultFileFormat() {
return ResultFileFormat.from(this.getVar(ConfVars.HIVEQUERYRESULTFILEFORMAT));
}
public interface EncoderDecoder<K, V> {
V encode(K key);
K decode(V value);
}
public static class URLEncoderDecoder implements EncoderDecoder<String, String> {
@Override
public String encode(String key) {
try {
return URLEncoder.encode(key, StandardCharsets.UTF_8.name());
} catch (UnsupportedEncodingException e) {
return key;
}
}
@Override
public String decode(String value) {
try {
return URLDecoder.decode(value, StandardCharsets.UTF_8.name());
} catch (UnsupportedEncodingException e) {
return value;
}
}
}
public static class EncoderDecoderFactory {
public static final URLEncoderDecoder URL_ENCODER_DECODER = new URLEncoderDecoder();
}
static {
ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
if (classLoader == null) {
classLoader = HiveConf.class.getClassLoader();
}
hiveDefaultURL = classLoader.getResource("hive-default.xml");
// Look for hive-site.xml on the CLASSPATH and log its location if found.
hiveSiteURL = findConfigFile(classLoader, "hive-site.xml", true);
hivemetastoreSiteUrl = findConfigFile(classLoader, "hivemetastore-site.xml", false);
hiveServer2SiteUrl = findConfigFile(classLoader, "hiveserver2-site.xml", false);
for (ConfVars confVar : ConfVars.values()) {
vars.put(confVar.varname, confVar);
}
Set<String> llapDaemonConfVarsSetLocal = new LinkedHashSet<>();
populateLlapDaemonVarsSet(llapDaemonConfVarsSetLocal);
llapDaemonVarsSet = Collections.unmodifiableSet(llapDaemonConfVarsSetLocal);
}
private static URL findConfigFile(ClassLoader classLoader, String name, boolean doLog) {
URL result = classLoader.getResource(name);
if (result == null) {
String confPath = System.getenv("HIVE_CONF_DIR");
result = checkConfigFile(new File(confPath, name));
if (result == null) {
String homePath = System.getenv("HIVE_HOME");
String nameInConf = "conf" + File.separator + name;
result = checkConfigFile(new File(homePath, nameInConf));
if (result == null) {
try {
// Handle both file:// and jar:<url>!{entry} in the case of shaded hive libs
URL sourceUrl = HiveConf.class.getProtectionDomain().getCodeSource().getLocation();
URI jarUri = sourceUrl.getProtocol().equalsIgnoreCase("jar") ? new URI(sourceUrl.getPath()) : sourceUrl.toURI();
// From the jar file, the parent is /lib folder
File parent = new File(jarUri).getParentFile();
if (parent != null) {
result = checkConfigFile(new File(parent.getParentFile(), nameInConf));
}
} catch (Throwable e) {
LOG.info("Cannot get jar URI", e);
System.err.println("Cannot get jar URI: " + e.getMessage());
}
}
}
}
if (doLog) {
LOG.info("Found configuration file {}", result);
}
return result;
}
private static URL checkConfigFile(File f) {
try {
return (f.exists() && f.isFile()) ? f.toURI().toURL() : null;
} catch (Throwable e) {
LOG.info("Error looking for config {}", f, e);
System.err.println("Error looking for config " + f + ": " + e.getMessage());
return null;
}
}
@InterfaceAudience.Private
public static final String PREFIX_LLAP = "llap.";
@InterfaceAudience.Private
public static final String PREFIX_HIVE_LLAP = "hive.llap.";
/**
* Metastore related options that the db is initialized against. When a conf
* var in this is list is changed, the metastore instance for the CLI will
* be recreated so that the change will take effect.
*/
public static final HiveConf.ConfVars[] metaVars = {
HiveConf.ConfVars.METASTOREWAREHOUSE,
HiveConf.ConfVars.REPLDIR,
HiveConf.ConfVars.METASTOREURIS,
HiveConf.ConfVars.METASTORESELECTION,
HiveConf.ConfVars.METASTORE_SERVER_PORT,
HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES,
HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES,
HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY,
HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT,
HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME,
HiveConf.ConfVars.METASTOREPWD,
HiveConf.ConfVars.METASTORECONNECTURLHOOK,
HiveConf.ConfVars.METASTORECONNECTURLKEY,
HiveConf.ConfVars.METASTORESERVERMINTHREADS,
HiveConf.ConfVars.METASTORESERVERMAXTHREADS,
HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE,
HiveConf.ConfVars.METASTORE_INT_ORIGINAL,
HiveConf.ConfVars.METASTORE_INT_ARCHIVED,
HiveConf.ConfVars.METASTORE_INT_EXTRACTED,
HiveConf.ConfVars.METASTORE_KERBEROS_KEYTAB_FILE,
HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL,
HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL,
HiveConf.ConfVars.METASTORE_TOKEN_SIGNATURE,
HiveConf.ConfVars.METASTORE_CACHE_PINOBJTYPES,
HiveConf.ConfVars.METASTORE_CONNECTION_POOLING_TYPE,
HiveConf.ConfVars.METASTORE_VALIDATE_TABLES,
HiveConf.ConfVars.METASTORE_DATANUCLEUS_INIT_COL_INFO,
HiveConf.ConfVars.METASTORE_VALIDATE_COLUMNS,
HiveConf.ConfVars.METASTORE_VALIDATE_CONSTRAINTS,
HiveConf.ConfVars.METASTORE_STORE_MANAGER_TYPE,
HiveConf.ConfVars.METASTORE_AUTO_CREATE_ALL,
HiveConf.ConfVars.METASTORE_TRANSACTION_ISOLATION,
HiveConf.ConfVars.METASTORE_CACHE_LEVEL2,
HiveConf.ConfVars.METASTORE_CACHE_LEVEL2_TYPE,
HiveConf.ConfVars.METASTORE_IDENTIFIER_FACTORY,
HiveConf.ConfVars.METASTORE_PLUGIN_REGISTRY_BUNDLE_CHECK,
HiveConf.ConfVars.METASTORE_AUTHORIZATION_STORAGE_AUTH_CHECKS,
HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_MAX,
HiveConf.ConfVars.METASTORE_EVENT_LISTENERS,
HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS,
HiveConf.ConfVars.METASTORE_EVENT_CLEAN_FREQ,
HiveConf.ConfVars.METASTORE_EVENT_EXPIRY_DURATION,
HiveConf.ConfVars.METASTORE_EVENT_MESSAGE_FACTORY,
HiveConf.ConfVars.METASTORE_FILTER_HOOK,
HiveConf.ConfVars.METASTORE_RAW_STORE_IMPL,
HiveConf.ConfVars.METASTORE_END_FUNCTION_LISTENERS,
HiveConf.ConfVars.METASTORE_PART_INHERIT_TBL_PROPS,
HiveConf.ConfVars.METASTORE_BATCH_RETRIEVE_OBJECTS_MAX,
HiveConf.ConfVars.METASTORE_INIT_HOOKS,
HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS,
HiveConf.ConfVars.HMSHANDLERATTEMPTS,
HiveConf.ConfVars.HMSHANDLERINTERVAL,
HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF,
HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN,
HiveConf.ConfVars.METASTORE_ORM_RETRIEVE_MAPNULLS_AS_EMPTY_STRINGS,
HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES,
HiveConf.ConfVars.USERS_IN_ADMIN_ROLE,
HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
HiveConf.ConfVars.HIVE_TXN_MANAGER,
HiveConf.ConfVars.HIVE_TXN_TIMEOUT,
HiveConf.ConfVars.HIVE_TXN_OPERATIONAL_PROPERTIES,
HiveConf.ConfVars.HIVE_TXN_HEARTBEAT_THREADPOOL_SIZE,
HiveConf.ConfVars.HIVE_TXN_MAX_OPEN_BATCH,
HiveConf.ConfVars.HIVE_TXN_RETRYABLE_SQLEX_REGEX,
HiveConf.ConfVars.HIVE_METASTORE_STATS_NDV_TUNER,
HiveConf.ConfVars.HIVE_METASTORE_STATS_NDV_DENSITY_FUNCTION,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_ENABLED,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_SIZE,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_PARTITIONS,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_FPP,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_VARIANCE,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_TTL,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_WRITER_WAIT,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_FULL,
HiveConf.ConfVars.METASTORE_AGGREGATE_STATS_CACHE_CLEAN_UNTIL,
HiveConf.ConfVars.METASTORE_FASTPATH,
HiveConf.ConfVars.METASTORE_HBASE_FILE_METADATA_THREADS,
HiveConf.ConfVars.METASTORE_WM_DEFAULT_POOL_SIZE
};
/**
* User configurable Metastore vars
*/
static final HiveConf.ConfVars[] metaConfVars = {
HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL,
HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL_DDL,
HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT,
HiveConf.ConfVars.METASTORE_PARTITION_NAME_WHITELIST_PATTERN,
HiveConf.ConfVars.METASTORE_CAPABILITY_CHECK,
HiveConf.ConfVars.METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES
};
static {
for (ConfVars confVar : metaConfVars) {
metaConfs.put(confVar.varname, confVar);
}
}
public static final String HIVE_LLAP_DAEMON_SERVICE_PRINCIPAL_NAME = "hive.llap.daemon.service.principal";
public static final String HIVE_SERVER2_AUTHENTICATION_LDAP_USERMEMBERSHIPKEY_NAME =
"hive.server2.authentication.ldap.userMembershipKey";
/**
* dbVars are the parameters can be set per database. If these
* parameters are set as a database property, when switching to that
* database, the HiveConf variable will be changed. The change of these
* parameters will effectively change the DFS and MapReduce clusters
* for different databases.
*/
public static final HiveConf.ConfVars[] dbVars = {
HiveConf.ConfVars.HADOOPBIN,
HiveConf.ConfVars.METASTOREWAREHOUSE,
HiveConf.ConfVars.SCRATCHDIR
};
/**
* encoded parameter values are ;-) encoded. Use decoder to get ;-) decoded string
*/
static final HiveConf.ConfVars[] ENCODED_CONF = {
ConfVars.HIVEQUERYSTRING
};
/**
* Variables used by LLAP daemons.
* TODO: Eventually auto-populate this based on prefixes. The conf variables
* will need to be renamed for this.
*/
private static final Set<String> llapDaemonVarsSet;
private static void populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal) {
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_ENABLED.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_MEMORY_MODE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ALLOCATOR_MIN_ALLOC.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ALLOCATOR_MAX_ALLOC.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ALLOCATOR_ARENA_COUNT.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_MEMORY_MAX_SIZE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ALLOCATOR_DIRECT.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_USE_LRFU.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_LRFU_LAMBDA.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_LRFU_BP_WRAPPER_SIZE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_CACHE_ALLOW_SYNTHETIC_FILEID.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_USE_FILEID_PATH.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_DECODING_METRICS_PERCENTILE_INTERVALS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ORC_ENABLE_TIME_COUNTERS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_THREADPOOL_SIZE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_KERBEROS_PRINCIPAL.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_KERBEROS_KEYTAB_FILE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ZKSM_ZK_CONNECTION_STRING.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_SECURITY_ACL.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_MANAGEMENT_ACL.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_SECURITY_ACL_DENY.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_MANAGEMENT_ACL_DENY.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DELEGATION_TOKEN_LIFETIME.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_MANAGEMENT_RPC_PORT.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_WEB_AUTO_AUTH.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_RPC_NUM_HANDLERS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WORK_DIRS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_YARN_SHUFFLE_PORT.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_YARN_CONTAINER_MB.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_SHUFFLE_DIR_WATCHER_ENABLED.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_AM_LIVENESS_HEARTBEAT_INTERVAL_MS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_TIMEOUT_MS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_AM_LIVENESS_CONNECTION_SLEEP_BETWEEN_RETRIES_MS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_NUM_EXECUTORS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_RPC_PORT.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_MEMORY_PER_INSTANCE_MB.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_XMX_HEADROOM.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_VCPUS_PER_INSTANCE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_NUM_FILE_CLEANER_THREADS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_FILE_CLEANUP_DELAY_SECONDS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_SERVICE_HOSTS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_SERVICE_REFRESH_INTERVAL.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_ALLOW_PERMANENT_FNS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_DOWNLOAD_PERMANENT_FNS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_TASK_SCHEDULER_WAIT_QUEUE_SIZE.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WAIT_QUEUE_COMPARATOR_CLASS_NAME.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_TASK_SCHEDULER_ENABLE_PREEMPTION.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_TASK_PREEMPTION_METRICS_INTERVALS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WEB_PORT.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_WEB_SSL.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_CONTAINER_ID.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_VALIDATE_ACLS.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_LOGGER.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_DAEMON_AM_USE_FQDN.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_OUTPUT_FORMAT_ARROW.varname);
llapDaemonVarsSetLocal.add(ConfVars.LLAP_IO_PATH_CACHE_SIZE.varname);
}
/**
* Get a set containing configuration parameter names used by LLAP Server instances
* @return an unmodifiable set containing llap ConfVars
*/
public static final Set<String> getLlapDaemonConfVars() {
return llapDaemonVarsSet;
}
/**
* ConfVars.
*
* These are the default configuration properties for Hive. Each HiveConf
* object is initialized as follows:
*
* 1) Hadoop configuration properties are applied.
* 2) ConfVar properties with non-null values are overlayed.
* 3) hive-site.xml properties are overlayed.
* 4) System Properties and Manual Overrides are overlayed.
*
* WARNING: think twice before adding any Hadoop configuration properties
* with non-null values to this list as they will override any values defined
* in the underlying Hadoop configuration.
*/
public static enum ConfVars {
MSC_CACHE_ENABLED("hive.metastore.client.cache.v2.enabled", true,
"This property enables a Caffeine Cache for Metastore client"),
MSC_CACHE_MAX_SIZE("hive.metastore.client.cache.v2.maxSize", "1Gb", new SizeValidator(),
"Set the maximum size (number of bytes) of the metastore client cache (DEFAULT: 1GB). " +
"Only in effect when the cache is enabled"),
MSC_CACHE_RECORD_STATS("hive.metastore.client.cache.v2.recordStats", false,
"This property enables recording metastore client cache stats in DEBUG logs"),
// QL execution stuff
SCRIPTWRAPPER("hive.exec.script.wrapper", null, ""),
PLAN("hive.exec.plan", "", ""),
STAGINGDIR("hive.exec.stagingdir", ".hive-staging",
"Directory name that will be created inside table locations in order to support HDFS encryption. " +
"This is replaces ${hive.exec.scratchdir} for query results with the exception of read-only tables. " +
"In all cases ${hive.exec.scratchdir} is still used for other temporary files, such as job plans."),
SCRATCHDIR("hive.exec.scratchdir", "/tmp/hive",
"HDFS root scratch dir for Hive jobs which gets created with write all (733) permission. " +
"For each connecting user, an HDFS scratch dir: ${hive.exec.scratchdir}/<username> is created, " +
"with ${hive.scratch.dir.permission}."),
REPLDIR("hive.repl.rootdir","/user/${system:user.name}/repl/",
"HDFS root dir for all replication dumps."),
REPLCMENABLED("hive.repl.cm.enabled", false,
"Turn on ChangeManager, so delete files will go to cmrootdir."),
REPLCMDIR("hive.repl.cmrootdir","/user/${system:user.name}/cmroot/",
"Root dir for ChangeManager, used for deleted files."),
REPLCMRETIAN("hive.repl.cm.retain","10d",
new TimeValidator(TimeUnit.DAYS),
"Time to retain removed files in cmrootdir."),
REPLCMENCRYPTEDDIR("hive.repl.cm.encryptionzone.rootdir", ".cmroot",
"Root dir for ChangeManager if encryption zones are enabled, used for deleted files."),
REPLCMFALLBACKNONENCRYPTEDDIR("hive.repl.cm.nonencryptionzone.rootdir",
"",
"Root dir for ChangeManager for non encrypted paths if hive.repl.cmrootdir is encrypted."),
REPLCMINTERVAL("hive.repl.cm.interval","3600s",
new TimeValidator(TimeUnit.SECONDS),
"Interval for cmroot cleanup thread."),
REPL_HA_DATAPATH_REPLACE_REMOTE_NAMESERVICE("hive.repl.ha.datapath.replace.remote.nameservice", false,
"When HDFS is HA enabled and both source and target clusters are configured with same nameservice name," +
"enable this flag and provide a new unique logical name for representing the remote cluster " +
"nameservice using config " + "'hive.repl.ha.datapath.replace.remote.nameservice.name'."),
REPL_HA_DATAPATH_REPLACE_REMOTE_NAMESERVICE_NAME("hive.repl.ha.datapath.replace.remote.nameservice.name", null,
"When HDFS is HA enabled and both source and target clusters are configured with same nameservice name, " +
"use this config to provide a unique logical name for nameservice on the remote cluster (should " +
"be different from nameservice name on the local cluster)"),
REPL_FUNCTIONS_ROOT_DIR("hive.repl.replica.functions.root.dir","/user/${system:user.name}/repl/functions/",
"Root directory on the replica warehouse where the repl sub-system will store jars from the primary warehouse"),
REPL_APPROX_MAX_LOAD_TASKS("hive.repl.approx.max.load.tasks", 10000,
"Provide an approximation of the maximum number of tasks that should be executed before \n"
+ "dynamically generating the next set of tasks. The number is approximate as Hive \n"
+ "will stop at a slightly higher number, the reason being some events might lead to a \n"
+ "task increment that would cross the specified limit."),
REPL_PARTITIONS_DUMP_PARALLELISM("hive.repl.partitions.dump.parallelism",100,
"Number of threads that will be used to dump partition data information during repl dump."),
REPL_RUN_DATA_COPY_TASKS_ON_TARGET("hive.repl.run.data.copy.tasks.on.target", true,
"Indicates whether replication should run data copy tasks during repl load operation."),
REPL_DUMP_METADATA_ONLY("hive.repl.dump.metadata.only", false,
"Indicates whether replication dump only metadata information or data + metadata. \n"
+ "This config makes hive.repl.include.external.tables config ineffective."),
REPL_RETAIN_PREV_DUMP_DIR("hive.repl.retain.prev.dump.dir", false,
"If this is set to false, then all previously used dump-directories will be deleted after repl-dump. " +
"If true, a number of latest dump-directories specified by hive.repl.retain.prev.dump.dir.count will be retained"),
REPL_RETAIN_PREV_DUMP_DIR_COUNT("hive.repl.retain.prev.dump.dir.count", 3,
"Indicates maximum number of latest previously used dump-directories which would be retained when " +
"hive.repl.retain.prev.dump.dir is set to true"),
REPL_RETAIN_CUSTOM_LOCATIONS_FOR_DB_ON_TARGET("hive.repl.retain.custom.db.locations.on.target", true,
"Indicates if source database has custom warehouse locations, whether that should be retained on target as well"),
REPL_INCLUDE_MATERIALIZED_VIEWS("hive.repl.include.materialized.views", false,
"Indicates whether replication of materialized views is enabled."),
REPL_DUMP_SKIP_IMMUTABLE_DATA_COPY("hive.repl.dump.skip.immutable.data.copy", false,
"Indicates whether replication dump can skip copyTask and refer to \n"
+ " original path instead. This would retain all table and partition meta"),
REPL_DUMP_METADATA_ONLY_FOR_EXTERNAL_TABLE("hive.repl.dump.metadata.only.for.external.table",
true,
"Indicates whether external table replication dump only metadata information or data + metadata"),
REPL_BOOTSTRAP_ACID_TABLES("hive.repl.bootstrap.acid.tables", false,
"Indicates if repl dump should bootstrap the information about ACID tables along with \n"
+ "incremental dump for replication. It is recommended to keep this config parameter \n"
+ "as false always and should be set to true only via WITH clause of REPL DUMP \n"
+ "command. It should be set to true only once for incremental repl dump on \n"
+ "each of the existing replication policies after enabling acid tables replication."),
REPL_BOOTSTRAP_DUMP_OPEN_TXN_TIMEOUT("hive.repl.bootstrap.dump.open.txn.timeout", "1h",
new TimeValidator(TimeUnit.HOURS),
"Indicates the timeout for all transactions which are opened before triggering bootstrap REPL DUMP. "
+ "If these open transactions are not closed within the timeout value, then REPL DUMP will "
+ "forcefully abort those transactions and continue with bootstrap dump."),
REPL_BOOTSTRAP_DUMP_ABORT_WRITE_TXN_AFTER_TIMEOUT("hive.repl.bootstrap.dump.abort.write.txn.after.timeout",
true,
"Indicates whether to abort write transactions belonging to the db under replication while doing a" +
" bootstrap dump after the timeout configured by hive.repl.bootstrap.dump.open.txn.timeout. If set to false," +
" bootstrap dump will fail."),
//https://hadoop.apache.org/docs/stable/hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html#Running_as_the_superuser
REPL_ADD_RAW_RESERVED_NAMESPACE("hive.repl.add.raw.reserved.namespace", false,
"For TDE with same encryption keys on source and target, allow Distcp super user to access \n"
+ "the raw bytes from filesystem without decrypting on source and then encrypting on target."),
REPL_INCLUDE_EXTERNAL_TABLES("hive.repl.include.external.tables", true,
"Indicates if repl dump should include information about external tables. It should be \n"
+ "used in conjunction with 'hive.repl.dump.metadata.only' set to false. if 'hive.repl.dump.metadata.only' \n"
+ " is set to true then this config parameter has no effect as external table meta data is flushed \n"
+ " always by default. If this config parameter is enabled on an on-going replication policy which is in\n"
+ " incremental phase, then need to set 'hive.repl.bootstrap.external.tables' to true for the first \n"
+ " repl dump to bootstrap all external tables."),
REPL_BOOTSTRAP_EXTERNAL_TABLES("hive.repl.bootstrap.external.tables", false,
"Indicates if repl dump should bootstrap the information about external tables along with incremental \n"
+ "dump for replication. It is recommended to keep this config parameter as false always and should be \n"
+ "set to true only via WITH clause of REPL DUMP command. It should be used in conjunction with \n"
+ "'hive.repl.include.external.tables' when sets to true. If 'hive.repl.include.external.tables' is \n"
+ "set to false, then this config parameter has no effect. It should be set to true only once for \n"
+ "incremental repl dump on each existing replication policy after enabling external tables replication."),
REPL_EXTERNAL_TABLE_BASE_DIR("hive.repl.replica.external.table.base.dir", null,
"This is the fully qualified base directory on the target/replica warehouse under which data for "
+ "external tables is stored. This is relative base path and hence prefixed to the source "
+ "external table path on target cluster."),
REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK("hive.repl.external.warehouse.single.copy.task",
false, "Should create single copy task for all the external tables "
+ "within the database default location for external tables, Would require more memory "
+ "for preparing the initial listing, Should be used if the memory "
+ "requirements can be fulfilled. If any specific configuration needs to be passed for these copy task it can"
+ " be specified using the prefix hive.dbpath."),
REPL_EXTERNAL_WAREHOUSE_SINGLE_COPY_TASK_PATHS("hive.repl.external.warehouse.single.copy.task.paths",
"", "Comma separated list of paths for which single copy task shall be created for all the external tables "
+ "within the locations Would require more memory for preparing the initial listing, Should be used if the memory "
+ "requirements can be fulfilled. If the directory contains data not part of the database, that data would "
+ "also get copied, so only locations which contains tables only belonging to the same database should be "
+ "provided. This has no effect in case of table level replication or if hive.repl.bootstrap.external.tables "
+ "isn't enabled. If any specific configuration needs to be passed for these copy task it can be specified "
+ "using the prefix hive.dbpath."),
REPL_INCLUDE_AUTHORIZATION_METADATA("hive.repl.include.authorization.metadata", false,
"This configuration will enable security and authorization related metadata along "
+ "with the hive data and metadata replication. "),
REPL_AUTHORIZATION_PROVIDER_SERVICE("hive.repl.authorization.provider.service", "ranger",
"This configuration will define which service will provide the security and authorization "
+ "related metadata that needs to be replicated along "
+ "with the hive data and metadata replication. Set the configuration "
+ "hive.repl.include.authorization.metadata to false to disable "
+ "security policies being replicated "),
REPL_RANGER_HANDLE_DENY_POLICY_TARGET("hive.repl.handle.ranger.deny.policy",
false,
"Indicates whether ranger deny policy for target database should be handled automatically by hive or not."),
HIVE_REPL_FAILOVER_START("hive.repl.failover.start",false,
"A replication policy level config to indicate if user wants to initiate fail-over " +
"to replicate the database in reverse direction."),
REPL_RANGER_ADD_DENY_POLICY_TARGET("hive.repl.ranger.target.deny.policy",
true,
"This configuration will add a deny policy on the target database for all users except hive"
+ " to avoid any update to the target database. Effective only if hive.repl.handle.ranger.deny.policy is set" +
"to true."),
REPL_RANGER_CLIENT_READ_TIMEOUT("hive.repl.ranger.client.read.timeout", "300s",
new TimeValidator(TimeUnit.SECONDS), "Ranger client read timeout for Ranger REST API calls."),
REPL_INCLUDE_ATLAS_METADATA("hive.repl.include.atlas.metadata", false,
"Indicates if Atlas metadata should be replicated along with Hive data and metadata or not."),
REPL_ATLAS_ENDPOINT("hive.repl.atlas.endpoint", null,
"Atlas endpoint of the current cluster hive database is getting replicated from/to."),
REPL_ATLAS_REPLICATED_TO_DB("hive.repl.atlas.replicatedto", null,
"Target hive database name Atlas metadata of source hive database is being replicated to."),
REPL_ATLAS_CLIENT_READ_TIMEOUT("hive.repl.atlas.client.read.timeout", "7200s",
new TimeValidator(TimeUnit.SECONDS), "Atlas client read timeout for Atlas REST API calls."),
REPL_EXTERNAL_CLIENT_CONNECT_TIMEOUT("hive.repl.external.client.connect.timeout", "10s",
new TimeValidator(TimeUnit.SECONDS), "Client connect timeout for REST API calls to external service."),
REPL_SOURCE_CLUSTER_NAME("hive.repl.source.cluster.name", null,
"Name of the source cluster for the replication."),
REPL_TARGET_CLUSTER_NAME("hive.repl.target.cluster.name", null,
"Name of the target cluster for the replication."),
REPL_RETRY_INTIAL_DELAY("hive.repl.retry.initial.delay", "60s",
new TimeValidator(TimeUnit.SECONDS),
"Initial Delay before retry starts."),
REPL_RETRY_BACKOFF_COEFFICIENT("hive.repl.retry.backoff.coefficient", 1.2f,
"The backoff coefficient for exponential retry delay between retries. " +
"Previous Delay * Backoff Coefficient will determine the next retry interval"),
REPL_RETRY_JITTER("hive.repl.retry.jitter", "30s", new TimeValidator(TimeUnit.SECONDS),
"A random jitter to be applied to avoid all retries happening at the same time."),
REPL_RETRY_MAX_DELAY_BETWEEN_RETRIES("hive.repl.retry.max.delay.between.retries", "60m",
new TimeValidator(TimeUnit.MINUTES),
"Maximum allowed retry delay in minutes after including exponential backoff. " +
"If this limit is reached, retry will continue with this retry duration."),
REPL_RETRY_TOTAL_DURATION("hive.repl.retry.total.duration", "24h",
new TimeValidator(TimeUnit.HOURS),
"Total allowed retry duration in hours inclusive of all retries. Once this is exhausted, " +
"the policy instance will be marked as failed and will need manual intervention to restart."),
REPL_COPY_FILE_LIST_ITERATOR_RETRY("hive.repl.copy.file.list.iterator.retry", true,
"Determines whether writes happen with retry upon encountering filesystem errors for data-copy \n"
+ "iterator files. It should be disabled when we do not want retry on a per-line basis while writing \n"
+ "to the files and in cases when flushing capabilities are not available on the stream. If disabled, then retry \n"
+ "is only attempted during file creation, not for errors encountered while writing entries."),
REPL_LOAD_PARTITIONS_BATCH_SIZE("hive.repl.load.partitions.batch.size", 10000,
"Provide the maximum number of partitions of a table that will be batched together during \n"
+ "repl load. All the partitions in a batch will make a single metastore call to update the metadata. \n"
+ "The data for these partitions will be copied before copying the metadata batch. "),
REPL_LOAD_PARTITIONS_WITH_DATA_COPY_BATCH_SIZE("hive.repl.load.partitions.with.data.copy.batch.size",1000,
"Provide the maximum number of partitions of a table that will be batched together during \n"
+ "repl load. All the partitions in a batch will make a single metastore call to update the metadata. \n"
+ "The data for these partitions will be copied before copying the metadata batch. "),
REPL_PARALLEL_COPY_TASKS("hive.repl.parallel.copy.tasks",100,
"Provide the maximum number of parallel copy operation(distcp or regular copy) launched for a table \n"
+ "or partition. This will create at max 100 threads which will run copy in parallel for the data files at \n"
+ " table or partition level. If hive.exec.parallel \n"
+ "is set to true then max worker threads created for copy can be hive.exec.parallel.thread.number(determines \n"
+ "number of copy tasks in parallel) * hive.repl.parallel.copy.tasks "),
REPL_SNAPSHOT_DIFF_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshotdiff.copy",
false,"Use snapshot diff for copying data from source to "
+ "destination cluster for external table in distcp. If true it uses snapshot based distcp for all the paths "
+ "configured as part of hive.repl.external.warehouse.single.copy.task along with the external warehouse "
+ "default location."),
REPL_SNAPSHOT_OVERWRITE_TARGET_FOR_EXTERNAL_TABLE_COPY("hive.repl.externaltable.snapshot.overwrite.target",
true,"If this is enabled, in case the target is modified, when using snapshot for external table"
+ "data copy, the target data is overwritten and the modifications are removed and the copy is again "
+ "attempted using the snapshot based approach. If disabled, the replication will fail in case the target is "
+ "modified."),
REPL_STATS_TOP_EVENTS_COUNTS("hive.repl.stats.events.count", 5,
"Number of topmost expensive events that needs to be maintained per event type for the replication statistics." +
" Maximum permissible limit is 10."),
LOCALSCRATCHDIR("hive.exec.local.scratchdir",
"${system:java.io.tmpdir}" + File.separator + "${system:user.name}",
"Local scratch space for Hive jobs"),
DOWNLOADED_RESOURCES_DIR("hive.downloaded.resources.dir",
"${system:java.io.tmpdir}" + File.separator + "${hive.session.id}_resources",
"Temporary local directory for added resources in the remote file system."),
SCRATCHDIRPERMISSION("hive.scratch.dir.permission", "700",
"The permission for the user specific scratch directories that get created."),
SUBMITVIACHILD("hive.exec.submitviachild", false, ""),
SUBMITLOCALTASKVIACHILD("hive.exec.submit.local.task.via.child", true,
"Determines whether local tasks (typically mapjoin hashtable generation phase) runs in \n" +
"separate JVM (true recommended) or not. \n" +
"Avoids the overhead of spawning new JVM, but can lead to out-of-memory issues."),
SCRIPTERRORLIMIT("hive.exec.script.maxerrsize", 100000,
"Maximum number of bytes a script is allowed to emit to standard error (per map-reduce task). \n" +
"This prevents runaway scripts from filling logs partitions to capacity"),
ALLOWPARTIALCONSUMP("hive.exec.script.allow.partial.consumption", false,
"When enabled, this option allows a user script to exit successfully without consuming \n" +
"all the data from the standard input."),
STREAMREPORTERPERFIX("stream.stderr.reporter.prefix", "reporter:",
"Streaming jobs that log to standard error with this prefix can log counter or status information."),
STREAMREPORTERENABLED("stream.stderr.reporter.enabled", true,
"Enable consumption of status and counter messages for streaming jobs."),
COMPRESSRESULT("hive.exec.compress.output", false,
"This controls whether the final outputs of a query (to a local/HDFS file or a Hive table) is compressed. \n" +
"The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"),
COMPRESSINTERMEDIATE("hive.exec.compress.intermediate", false,
"This controls whether intermediate files produced by Hive between multiple map-reduce jobs are compressed. \n" +
"The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"),
COMPRESSINTERMEDIATECODEC("hive.intermediate.compression.codec", "", ""),
COMPRESSINTERMEDIATETYPE("hive.intermediate.compression.type", "", ""),
BYTESPERREDUCER("hive.exec.reducers.bytes.per.reducer", (long) (256 * 1000 * 1000),
"size per reducer.The default is 256Mb, i.e if the input size is 1G, it will use 4 reducers."),
MAXREDUCERS("hive.exec.reducers.max", 1009,
"max number of reducers will be used. If the one specified in the configuration parameter mapred.reduce.tasks is\n" +
"negative, Hive will use this one as the max number of reducers when automatically determine number of reducers."),
PREEXECHOOKS("hive.exec.pre.hooks", "",
"Comma-separated list of pre-execution hooks to be invoked for each statement. \n" +
"A pre-execution hook is specified as the name of a Java class which implements the \n" +
"org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."),
POSTEXECHOOKS("hive.exec.post.hooks", "",
"Comma-separated list of post-execution hooks to be invoked for each statement. \n" +
"A post-execution hook is specified as the name of a Java class which implements the \n" +
"org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."),
ONFAILUREHOOKS("hive.exec.failure.hooks", "",
"Comma-separated list of on-failure hooks to be invoked for each statement. \n" +
"An on-failure hook is specified as the name of Java class which implements the \n" +
"org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."),
QUERYREDACTORHOOKS("hive.exec.query.redactor.hooks", "",
"Comma-separated list of hooks to be invoked for each query which can \n" +
"transform the query before it's placed in the job.xml file. Must be a Java class which \n" +
"extends from the org.apache.hadoop.hive.ql.hooks.Redactor abstract class."),
CLIENTSTATSPUBLISHERS("hive.client.stats.publishers", "",
"Comma-separated list of statistics publishers to be invoked on counters on each job. \n" +
"A client stats publisher is specified as the name of a Java class which implements the \n" +
"org.apache.hadoop.hive.ql.stats.ClientStatsPublisher interface."),
BASICSTATSTASKSMAXTHREADSFACTOR("hive.basic.stats.max.threads.factor", 2, "Determines the maximum number of "
+ "threads that can be used for collection of file level statistics. If the value configured is x, then the "
+ "maximum number of threads that can be used is x multiplied by the number of available processors. A value"
+ " of less than 1, makes stats collection sequential."),
EXECPARALLEL("hive.exec.parallel", false, "Whether to execute jobs in parallel"),
EXECPARALLETHREADNUMBER("hive.exec.parallel.thread.number", 8,
"How many jobs at most can be executed in parallel"),
@Deprecated
HIVESPECULATIVEEXECREDUCERS("hive.mapred.reduce.tasks.speculative.execution", false,
"(Deprecated) Whether speculative execution for reducers should be turned on. "),
HIVECOUNTERSPULLINTERVAL("hive.exec.counters.pull.interval", 1000L,
"The interval with which to poll the JobTracker for the counters the running job. \n" +
"The smaller it is the more load there will be on the jobtracker, the higher it is the less granular the caught will be."),
DYNAMICPARTITIONING("hive.exec.dynamic.partition", true,
"Whether or not to allow dynamic partitions in DML/DDL."),
DYNAMICPARTITIONINGMODE("hive.exec.dynamic.partition.mode", "nonstrict",
new StringSet("strict", "nonstrict"),
"In strict mode, the user must specify at least one static partition\n" +
"in case the user accidentally overwrites all partitions.\n" +
"In nonstrict mode all partitions are allowed to be dynamic."),
DYNAMICPARTITIONMAXPARTS("hive.exec.max.dynamic.partitions", 1000,
"Maximum number of dynamic partitions allowed to be created in total."),
DYNAMICPARTITIONMAXPARTSPERNODE("hive.exec.max.dynamic.partitions.pernode", 100,
"Maximum number of dynamic partitions allowed to be created in each mapper/reducer node."),
DYNAMICPARTITIONCONVERT("hive.exec.dynamic.partition.type.conversion", true,
"Whether to check and cast a dynamic partition column before creating the partition " +
"directory. For example, if partition p is type int and we insert string '001', then if " +
"this value is true, directory p=1 will be created; if false, p=001"),
MAXCREATEDFILES("hive.exec.max.created.files", 100000L,
"Maximum number of HDFS files created by all mappers/reducers in a MapReduce job."),
DEFAULTPARTITIONNAME("hive.exec.default.partition.name", "__HIVE_DEFAULT_PARTITION__",
"The default partition name in case the dynamic partition column value is null/empty string or any other values that cannot be escaped. \n" +
"This value must not contain any special character used in HDFS URI (e.g., ':', '%', '/' etc). \n" +
"The user has to be aware that the dynamic partition value should not contain this value to avoid confusions."),
DEFAULT_ZOOKEEPER_PARTITION_NAME("hive.lockmgr.zookeeper.default.partition.name", "__HIVE_DEFAULT_ZOOKEEPER_PARTITION__", ""),
// Whether to show a link to the most failed task + debugging tips
SHOW_JOB_FAIL_DEBUG_INFO("hive.exec.show.job.failure.debug.info", true,
"If a job fails, whether to provide a link in the CLI to the task with the\n" +
"most failures, along with debugging hints if applicable."),
JOB_DEBUG_CAPTURE_STACKTRACES("hive.exec.job.debug.capture.stacktraces", true,
"Whether or not stack traces parsed from the task logs of a sampled failed task \n" +
"for each failed job should be stored in the SessionState"),
JOB_DEBUG_TIMEOUT("hive.exec.job.debug.timeout", 30000, ""),
TASKLOG_DEBUG_TIMEOUT("hive.exec.tasklog.debug.timeout", 20000, ""),
OUTPUT_FILE_EXTENSION("hive.output.file.extension", null,
"String used as a file extension for output files. \n" +
"If not set, defaults to the codec extension for text files (e.g. \".gz\"), or no extension otherwise."),
HIVE_IN_TEST("hive.in.test", false, "internal usage only, true in test mode", true),
HIVE_IN_TEST_ICEBERG("hive.in.iceberg.test", false, "internal usage only, true when " +
"testing iceberg", true),
HIVE_IN_TEST_SSL("hive.in.ssl.test", false, "internal usage only, true in SSL test mode", true),
// TODO: this needs to be removed; see TestReplicationScenarios* comments.
HIVE_IN_TEST_REPL("hive.in.repl.test", false, "internal usage only, true in replication test mode", true),
HIVE_IN_TEST_IDE("hive.in.ide.test", false, "internal usage only, true if test running in ide",
true),
HIVE_TESTING_SHORT_LOGS("hive.testing.short.logs", false,
"internal usage only, used only in test mode. If set true, when requesting the " +
"operation logs the short version (generated by LogDivertAppenderForTest) will be " +
"returned"),
HIVE_TESTING_REMOVE_LOGS("hive.testing.remove.logs", true,
"internal usage only, used only in test mode. If set false, the operation logs, and the " +
"operation log directory will not be removed, so they can be found after the test runs."),
HIVE_TEST_LOAD_HOSTNAMES("hive.test.load.hostnames", "",
"Specify host names for load testing. (e.g., \"host1,host2,host3\"). Leave it empty if no " +
"load generation is needed (eg. for production)."),
HIVE_TEST_LOAD_INTERVAL("hive.test.load.interval", "10ms", new TimeValidator(TimeUnit.MILLISECONDS),
"The interval length used for load and idle periods in milliseconds."),
HIVE_TEST_LOAD_UTILIZATION("hive.test.load.utilization", 0.2f,
"Specify processor load utilization between 0.0 (not loaded on all threads) and 1.0 " +
"(fully loaded on all threads). Comparing this with a random value the load generator creates " +
"hive.test.load.interval length active loops or idle periods"),
HIVE_IN_TEZ_TEST("hive.in.tez.test", false, "internal use only, true when in testing tez",
true),
HIVE_MAPJOIN_TESTING_NO_HASH_TABLE_LOAD("hive.mapjoin.testing.no.hash.table.load", false, "internal use only, true when in testing map join",
true),
HIVE_ADDITIONAL_PARTIAL_MASKS_PATTERN("hive.qtest.additional.partial.mask.pattern", "",
"internal use only, used in only qtests. Provide additional partial masks pattern" +
"for qtests as a ',' separated list"),
HIVE_ADDITIONAL_PARTIAL_MASKS_REPLACEMENT_TEXT("hive.qtest.additional.partial.mask.replacement.text", "",
"internal use only, used in only qtests. Provide additional partial masks replacement" +
"text for qtests as a ',' separated list"),
HIVE_IN_REPL_TEST_FILES_SORTED("hive.in.repl.test.files.sorted", false,
"internal usage only, set to true if the file listing is required in sorted order during bootstrap load", true),
LOCALMODEAUTO("hive.exec.mode.local.auto", false,
"Let Hive determine whether to run in local mode automatically"),
LOCALMODEMAXBYTES("hive.exec.mode.local.auto.inputbytes.max", 134217728L,
"When hive.exec.mode.local.auto is true, input bytes should less than this for local mode."),
LOCALMODEMAXINPUTFILES("hive.exec.mode.local.auto.input.files.max", 4,
"When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode."),
DROP_IGNORES_NON_EXISTENT("hive.exec.drop.ignorenonexistent", true,
"Do not report an error if DROP TABLE/VIEW/Index/Function specifies a nonexistent table/view/function"),
HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"),
HIVE_FILE_MAX_FOOTER("hive.file.max.footer", 100,
"maximum number of lines for footer user can define for a table file"),
HIVE_RESULTSET_USE_UNIQUE_COLUMN_NAMES("hive.resultset.use.unique.column.names", true,
"Make column names unique in the result set by qualifying column names with table alias if needed.\n" +
"Table alias will be added to column names for queries of type \"select *\" or \n" +
"if query explicitly uses table alias \"select r1.x..\"."),
HIVE_PROTO_EVENTS_QUEUE_CAPACITY("hive.hook.proto.queue.capacity", 64,
"Queue capacity for the proto events logging threads."),
HIVE_PROTO_EVENTS_BASE_PATH("hive.hook.proto.base-directory", "",
"Base directory into which the proto event messages are written by HiveProtoLoggingHook."),
HIVE_PROTO_EVENTS_ROLLOVER_CHECK_INTERVAL("hive.hook.proto.rollover-interval", "600s",
new TimeValidator(TimeUnit.SECONDS, 0L, true, 3600 * 24L, true),
"Frequency at which the file rollover check is triggered."),
HIVE_PROTO_EVENTS_CLEAN_FREQ("hive.hook.proto.events.clean.freq", "1d",
new TimeValidator(TimeUnit.DAYS),
"Frequency at which timer task runs to purge expired proto event files."),
HIVE_PROTO_EVENTS_TTL("hive.hook.proto.events.ttl", "7d",
new TimeValidator(TimeUnit.DAYS),
"Time-To-Live (TTL) of proto event files before cleanup."),
HIVE_PROTO_FILE_PER_EVENT("hive.hook.proto.file.per.event", false,
"Whether each proto event has to be written to separate file. " +
"(Use this for FS that does not hflush immediately like S3A)"),
// Hadoop Configuration Properties
// Properties with null values are ignored and exist only for the purpose of giving us
// a symbolic name to reference in the Hive source code. Properties with non-null
// values will override any values set in the underlying Hadoop configuration.
HADOOPBIN("hadoop.bin.path", findHadoopBinary(), "", true),
YARNBIN("yarn.bin.path", findYarnBinary(), "", true),
MAPREDBIN("mapred.bin.path", findMapRedBinary(), "", true),
HIVE_FS_HAR_IMPL("fs.har.impl", "org.apache.hadoop.hive.shims.HiveHarFileSystem",
"The implementation for accessing Hadoop Archives. Note that this won't be applicable to Hadoop versions less than 0.20"),
MAPREDMAXSPLITSIZE(FileInputFormat.SPLIT_MAXSIZE, 256000000L, "", true),
MAPREDMINSPLITSIZE(FileInputFormat.SPLIT_MINSIZE, 1L, "", true),
MAPREDMINSPLITSIZEPERNODE(CombineFileInputFormat.SPLIT_MINSIZE_PERNODE, 1L, "", true),
MAPREDMINSPLITSIZEPERRACK(CombineFileInputFormat.SPLIT_MINSIZE_PERRACK, 1L, "", true),
// The number of reduce tasks per job. Hadoop sets this value to 1 by default
// By setting this property to -1, Hive will automatically determine the correct
// number of reducers.
HADOOPNUMREDUCERS("mapreduce.job.reduces", -1, "", true),
// Metastore stuff. Be sure to update HiveConf.metaVars when you add something here!
METASTOREDBTYPE("hive.metastore.db.type", "DERBY", new StringSet("DERBY", "ORACLE", "MYSQL", "MSSQL", "POSTGRES"),
"Type of database used by the metastore. Information schema & JDBCStorageHandler depend on it."),
/**
* @deprecated Use MetastoreConf.WAREHOUSE
*/
@Deprecated
METASTOREWAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse",
"location of default database for the warehouse"),
HIVE_METASTORE_WAREHOUSE_EXTERNAL("hive.metastore.warehouse.external.dir", null,
"Default location for external tables created in the warehouse. " +
"If not set or null, then the normal warehouse location will be used as the default location."),
/**
* @deprecated Use MetastoreConf.THRIFT_URIS
*/
@Deprecated
METASTOREURIS("hive.metastore.uris", "",
"Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore."),
/**
* @deprecated Use MetastoreConf.THRIFT_URI_SELECTION
*/
@Deprecated
METASTORESELECTION("hive.metastore.uri.selection", "RANDOM",
new StringSet("SEQUENTIAL", "RANDOM"),
"Determines the selection mechanism used by metastore client to connect to remote " +
"metastore. SEQUENTIAL implies that the first valid metastore from the URIs specified " +
"as part of hive.metastore.uris will be picked. RANDOM implies that the metastore " +
"will be picked randomly"),
/**
* @deprecated Use MetastoreConf.CAPABILITY_CHECK
*/
@Deprecated
METASTORE_CAPABILITY_CHECK("hive.metastore.client.capability.check", true,
"Whether to check client capabilities for potentially breaking API usage."),
METASTORE_CLIENT_CAPABILITIES("hive.metastore.client.capabilities", "", "Capabilities possessed by HiveServer"),
METASTORE_CLIENT_CACHE_ENABLED("hive.metastore.client.cache.enabled", false,
"Whether to enable metastore client cache"),
METASTORE_CLIENT_CACHE_EXPIRY_TIME("hive.metastore.client.cache.expiry.time", "120s",
new TimeValidator(TimeUnit.SECONDS), "Expiry time for metastore client cache"),
METASTORE_CLIENT_CACHE_INITIAL_CAPACITY("hive.metastore.client.cache.initial.capacity", 50,
"Initial capacity for metastore client cache"),
METASTORE_CLIENT_CACHE_MAX_CAPACITY("hive.metastore.client.cache.max.capacity", 50,
"Max capacity for metastore client cache"),
METASTORE_CLIENT_CACHE_STATS_ENABLED("hive.metastore.client.cache.stats.enabled", false,
"Whether to enable metastore client cache stats"),
METASTORE_FASTPATH("hive.metastore.fastpath", false,
"Used to avoid all of the proxies and object copies in the metastore. Note, if this is " +
"set, you MUST use a local metastore (hive.metastore.uris must be empty) otherwise " +
"undefined and most likely undesired behavior will result"),
/**
* @deprecated Use MetastoreConf.FS_HANDLER_THREADS_COUNT
*/
@Deprecated
METASTORE_FS_HANDLER_THREADS_COUNT("hive.metastore.fshandler.threads", 15,
"Number of threads to be allocated for metastore handler for fs operations."),
/**
* @deprecated Use MetastoreConf.FILE_METADATA_THREADS
*/
@Deprecated
METASTORE_HBASE_FILE_METADATA_THREADS("hive.metastore.hbase.file.metadata.threads", 1,
"Number of threads to use to read file metadata in background to cache it."),
/**
* @deprecated Use MetastoreConf.URI_RESOLVER
*/
@Deprecated
METASTORE_URI_RESOLVER("hive.metastore.uri.resolver", "",
"If set, fully qualified class name of resolver for hive metastore uri's"),
/**
* @deprecated Use MetastoreConf.THRIFT_CONNECTION_RETRIES
*/
@Deprecated
METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3,
"Number of retries while opening a connection to metastore"),
/**
* @deprecated Use MetastoreConf.THRIFT_FAILURE_RETRIES
*/
@Deprecated
METASTORETHRIFTFAILURERETRIES("hive.metastore.failure.retries", 1,
"Number of retries upon failure of Thrift metastore calls"),
/**
* @deprecated Use MetastoreConf.SERVER_PORT
*/
@Deprecated
METASTORE_SERVER_PORT("hive.metastore.port", 9083, "Hive metastore listener port"),
/**
* @deprecated Use MetastoreConf.CLIENT_CONNECT_RETRY_DELAY
*/
@Deprecated
METASTORE_CLIENT_CONNECT_RETRY_DELAY("hive.metastore.client.connect.retry.delay", "1s",
new TimeValidator(TimeUnit.SECONDS),
"Number of seconds for the client to wait between consecutive connection attempts"),
/**
* @deprecated Use MetastoreConf.CLIENT_SOCKET_TIMEOUT
*/
@Deprecated
METASTORE_CLIENT_SOCKET_TIMEOUT("hive.metastore.client.socket.timeout", "600s",
new TimeValidator(TimeUnit.SECONDS),
"MetaStore Client socket timeout in seconds"),
/**
* @deprecated Use MetastoreConf.CLIENT_SOCKET_LIFETIME
*/
@Deprecated
METASTORE_CLIENT_SOCKET_LIFETIME("hive.metastore.client.socket.lifetime", "0s",
new TimeValidator(TimeUnit.SECONDS),
"MetaStore Client socket lifetime in seconds. After this time is exceeded, client\n" +
"reconnects on the next MetaStore operation. A value of 0s means the connection\n" +
"has an infinite lifetime."),
/**