-
Notifications
You must be signed in to change notification settings - Fork 6
/
BeeLineOpts.java
719 lines (587 loc) · 18.8 KB
/
BeeLineOpts.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
/*
* 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.
*/
/*
* This source file is based on code taken from SQLLine 1.0.2
* See SQLLine notice in LICENSE
*/
package org.apache.hive.beeline;
import java.io.File;
import java.io.FileInputStream;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.Arrays;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.TreeSet;
import jline.Terminal;
import jline.TerminalFactory;
import jline.console.completer.Completer;
import jline.console.completer.StringsCompleter;
import jline.console.history.MemoryHistory;
import org.apache.hadoop.hive.conf.HiveConf;
class BeeLineOpts implements Completer {
public static final int DEFAULT_MAX_WIDTH = 80;
public static final int DEFAULT_MAX_HEIGHT = 80;
public static final int DEFAULT_HEADER_INTERVAL = 100;
public static final String DEFAULT_ISOLATION_LEVEL =
"TRANSACTION_REPEATABLE_READ";
public static final String PROPERTY_PREFIX = "beeline.";
public static final String PROPERTY_NAME_EXIT =
PROPERTY_PREFIX + "system.exit";
public static final String DEFAULT_NULL_STRING = "NULL";
public static final char DEFAULT_DELIMITER_FOR_DSV = '|';
public static final int DEFAULT_MAX_COLUMN_WIDTH = 50;
public static final int DEFAULT_INCREMENTAL_BUFFER_ROWS = 1000;
public static final String DEFAULT_DELIMITER = ";";
public static final String URL_ENV_PREFIX = "BEELINE_URL_";
private final BeeLine beeLine;
private boolean autosave = false;
private boolean silent = false;
private boolean color = false;
private boolean showHeader = true;
private boolean escapeCRLF = false;
private boolean showDbInPrompt = false;
private int headerInterval = 100;
private boolean fastConnect = true;
private boolean autoCommit = true;
private boolean verbose = false;
private boolean force = false;
private boolean incremental = true;
private boolean convertBinaryArrayToString = true;
private int incrementalBufferRows = DEFAULT_INCREMENTAL_BUFFER_ROWS;
private boolean showWarnings = false;
private boolean showNestedErrs = false;
private boolean showElapsedTime = true;
private boolean entireLineAsCommand = false;
private String numberFormat = "default";
private final Terminal terminal = TerminalFactory.get();
private int maxWidth = DEFAULT_MAX_WIDTH;
private int maxHeight = DEFAULT_MAX_HEIGHT;
private int maxColumnWidth = DEFAULT_MAX_COLUMN_WIDTH;
int timeout = -1;
private String isolation = DEFAULT_ISOLATION_LEVEL;
private String outputFormat = "table";
// This configuration is used only for client side configuration.
private HiveConf conf;
private boolean trimScripts = true;
private boolean allowMultiLineCommand = true;
//This can be set for old behavior of nulls printed as empty strings
private boolean nullEmptyString = false;
private boolean truncateTable = false;
private final File rcFile = new File(saveDir(), "beeline.properties");
private String historyFile = new File(saveDir(), "history").getAbsolutePath();
private int maxHistoryRows = MemoryHistory.DEFAULT_MAX_SIZE;
private String scriptFile = null;
private String[] initFiles = null;
private String authType = null;
private char delimiterForDSV = DEFAULT_DELIMITER_FOR_DSV;
private Map<String, String> hiveVariables = new HashMap<String, String>();
private Map<String, String> hiveConfVariables = new HashMap<String, String>();
private boolean helpAsked;
private boolean beelineSiteUrlsAsked;
private String lastConnectedUrl = null;
private TreeSet<String> cachedPropertyNameSet = null;
private String delimiter = DEFAULT_DELIMITER;
@Retention(RetentionPolicy.RUNTIME)
public @interface Ignore {
// marker annotations for functions that Reflector should ignore / pretend it does not exist
// NOTE: BeeLineOpts uses Reflector in an extensive way to call getters and setters on itself
// If you want to add any getters or setters to this class, but not have it interfere with
// saved variables in beeline.properties, careful use of this marker is needed.
// Also possible to get this by naming these functions obtainBlah instead of getBlah
// and so on, but that is not explicit and will likely surprise people looking at the
// code in the future. Better to be explicit in intent.
}
public interface Env {
// Env interface to mock out dealing with Environment variables
// This allows us to interface with Environment vars through
// BeeLineOpts while allowing tests to mock out Env setting if needed.
String get(String envVar);
}
public static Env env = new Env() {
@Override
public String get(String envVar) {
return System.getenv(envVar); // base env impl simply defers to System.getenv.
}
};
public BeeLineOpts(BeeLine beeLine, Properties props) {
this.beeLine = beeLine;
if (terminal.getWidth() > 0) {
maxWidth = terminal.getWidth();
}
if (terminal.getHeight() > 0) {
maxHeight = terminal.getHeight();
}
loadProperties(props);
}
public Completer[] optionCompleters() {
return new Completer[] {this};
}
public String[] possibleSettingValues() {
List<String> vals = new LinkedList<String>();
vals.addAll(Arrays.asList(new String[] { "yes", "no" }));
return vals.toArray(new String[vals.size()]);
}
/**
* The save directory if HOME/.beeline/ on UNIX, and
* HOME/beeline/ on Windows.
*/
public File saveDir() {
String dir = System.getProperty("beeline.rcfile");
if (dir != null && dir.length() > 0) {
return new File(dir);
}
File f = new File(System.getProperty("user.home"),
(System.getProperty("os.name").toLowerCase()
.indexOf("windows") != -1 ? "" : ".") + "beeline")
.getAbsoluteFile();
try {
f.mkdirs();
} catch (Exception e) {
}
return f;
}
@Override
public int complete(String buf, int pos, List cand) {
try {
return new StringsCompleter(propertyNames()).complete(buf, pos, cand);
} catch (Exception e) {
beeLine.handleException(e);
return -1;
}
}
public void save() throws IOException {
try (OutputStream out = new FileOutputStream(rcFile)) {
save(out);
}
}
public void save(OutputStream out) throws IOException {
try {
Properties props = toProperties();
// don't save maxwidth: it is automatically set based on
// the terminal configuration
props.remove(PROPERTY_PREFIX + "maxwidth");
props.store(out, beeLine.getApplicationTitle());
} catch (Exception e) {
beeLine.handleException(e);
}
}
String[] propertyNames()
throws IllegalAccessException, InvocationTargetException {
Set<String> names = propertyNamesSet(); // make sure we initialize if necessary
return names.toArray(new String[names.size()]);
}
Set<String> propertyNamesSet()
throws IllegalAccessException, InvocationTargetException {
if (cachedPropertyNameSet == null){
TreeSet<String> names = new TreeSet<String>();
// get all the values from getXXX methods
Method[] m = getClass().getDeclaredMethods();
for (int i = 0; m != null && i < m.length; i++) {
if (!(m[i].getName().startsWith("get"))) {
continue;
}
if (m[i].getAnnotation(Ignore.class) != null){
continue; // not actually a getter
}
if (m[i].getParameterTypes().length != 0) {
continue;
}
String propName = m[i].getName().substring(3).toLowerCase();
names.add(propName);
}
cachedPropertyNameSet = names;
}
return cachedPropertyNameSet;
}
public Properties toProperties()
throws IllegalAccessException, InvocationTargetException,
ClassNotFoundException {
Properties props = new Properties();
String[] names = propertyNames();
for (int i = 0; names != null && i < names.length; i++) {
Object o = beeLine.getReflector().invoke(this, "get" + names[i], new Object[0]);
props.setProperty(PROPERTY_PREFIX + names[i],
o == null ? "" : o.toString());
}
beeLine.debug("properties: " + props.toString());
return props;
}
public void load() throws IOException {
try (InputStream in = new FileInputStream(rcFile)) {
load(in);
}
}
public void load(InputStream fin) throws IOException {
Properties p = new Properties();
p.load(fin);
loadProperties(p);
}
/**
* Update the options after connection is established in CLI mode.
*/
public void updateBeeLineOptsFromConf() {
if (!beeLine.isBeeLine()) {
if (conf == null) {
conf = beeLine.getCommands().getHiveConf(false);
}
setForce(HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIIGNOREERRORS));
}
}
public void setHiveConf(HiveConf conf) {
this.conf = conf;
}
public void loadProperties(Properties props) {
for (Object element : props.keySet()) {
String key = element.toString();
if (key.equals(PROPERTY_NAME_EXIT)) {
// fix for sf.net bug 879422
continue;
}
if (key.startsWith(PROPERTY_PREFIX)) {
set(key.substring(PROPERTY_PREFIX.length()),
props.getProperty(key));
}
}
}
public void set(String key, String value) {
set(key, value, false);
}
public boolean set(String key, String value, boolean quiet) {
try {
beeLine.getReflector().invoke(this, "set" + key, new Object[] {value});
return true;
} catch (Exception e) {
if (!quiet) {
beeLine.error(beeLine.loc("error-setting", new Object[] {key, e}));
}
return false;
}
}
public void setFastConnect(boolean fastConnect) {
this.fastConnect = fastConnect;
}
public String getAuthType() {
return authType;
}
public void setAuthType(String authType) {
this.authType = authType;
}
public boolean getFastConnect() {
return fastConnect;
}
public void setAutoCommit(boolean autoCommit) {
this.autoCommit = autoCommit;
}
public boolean getAutoCommit() {
return autoCommit;
}
public void setVerbose(boolean verbose) {
this.verbose = verbose;
}
public boolean getVerbose() {
return verbose;
}
public void setShowWarnings(boolean showWarnings) {
this.showWarnings = showWarnings;
}
public boolean getShowWarnings() {
return showWarnings;
}
public void setShowNestedErrs(boolean showNestedErrs) {
this.showNestedErrs = showNestedErrs;
}
public boolean getShowNestedErrs() {
return showNestedErrs;
}
public void setShowElapsedTime(boolean showElapsedTime) {
this.showElapsedTime = showElapsedTime;
}
public boolean getShowElapsedTime() {
return showElapsedTime;
}
public void setNumberFormat(String numberFormat) {
this.numberFormat = numberFormat;
}
public String getNumberFormat() {
return numberFormat;
}
public void setConvertBinaryArrayToString(boolean convert) {
this.convertBinaryArrayToString = convert;
}
public boolean getConvertBinaryArrayToString() {
return this.convertBinaryArrayToString;
}
public void setMaxWidth(int maxWidth) {
this.maxWidth = maxWidth;
}
public int getMaxWidth() {
return maxWidth;
}
public void setMaxColumnWidth(int maxColumnWidth) {
this.maxColumnWidth = maxColumnWidth;
}
public int getMaxColumnWidth() {
return maxColumnWidth;
}
public void setTimeout(int timeout) {
this.timeout = timeout;
}
public int getTimeout() {
return timeout;
}
public void setIsolation(String isolation) {
this.isolation = isolation;
}
public String getIsolation() {
return isolation;
}
public void setEntireLineAsCommand(boolean entireLineAsCommand) {
this.entireLineAsCommand = entireLineAsCommand;
}
public boolean getEntireLineAsCommand() {
return entireLineAsCommand;
}
public void setHistoryFile(String historyFile) {
this.historyFile = historyFile;
}
public String getHistoryFile() {
return historyFile;
}
/**
* @param numRows - the number of rows to store in history file
*/
public void setMaxHistoryRows(int numRows) {
this.maxHistoryRows = numRows;
}
public int getMaxHistoryRows() {
return maxHistoryRows;
}
public void setScriptFile(String scriptFile) {
this.scriptFile = scriptFile;
}
public String getScriptFile() {
return scriptFile;
}
public String[] getInitFiles() {
return initFiles;
}
public void setInitFiles(String[] initFiles) {
this.initFiles = initFiles;
}
public void setColor(boolean color) {
this.color = color;
}
public boolean getColor() {
return color;
}
public void setShowHeader(boolean showHeader) {
this.showHeader = showHeader;
}
public boolean getShowHeader() {
if (beeLine.isBeeLine()) {
return showHeader;
} else {
boolean header;
HiveConf conf = beeLine.getCommands().getHiveConf(true);
header = HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER);
return header;
}
}
public void setEscapeCRLF(boolean escapeCRLF) {
this.escapeCRLF = escapeCRLF;
}
public boolean getEscapeCRLF() {
if (beeLine.isBeeLine()) {
return escapeCRLF;
} else { //hive cli
if(conf != null) {
return HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_ESCAPE_CRLF);
} else {
return false;
}
}
}
public void setShowDbInPrompt(boolean showDbInPrompt) {
this.showDbInPrompt = showDbInPrompt;
}
/**
* In beeline mode returns the beeline option provided by command line argument or config file
* In compatibility mode returns the value of the hive.cli.print.current.db config variable
* @return Should the current db displayed in the prompt
*/
public boolean getShowDbInPrompt() {
if (beeLine.isBeeLine()) {
return showDbInPrompt;
} else {
HiveConf conf = beeLine.getCommands().getHiveConf(true);
return HiveConf.getBoolVar(conf, HiveConf.ConfVars.CLIPRINTCURRENTDB);
}
}
public void setHeaderInterval(int headerInterval) {
this.headerInterval = headerInterval;
}
public int getHeaderInterval() {
return headerInterval;
}
public void setForce(boolean force) {
this.force = force;
}
public boolean getForce() {
return force;
}
public void setIncremental(boolean incremental) {
this.incremental = incremental;
}
public boolean getIncremental() {
return incremental;
}
public void setIncrementalBufferRows(int incrementalBufferRows) {
this.incrementalBufferRows = incrementalBufferRows;
}
public int getIncrementalBufferRows() {
return this.incrementalBufferRows;
}
public void setSilent(boolean silent) {
this.silent = silent;
}
public boolean isSilent() {
return silent;
}
public void setAutosave(boolean autosave) {
this.autosave = autosave;
}
public boolean getAutosave() {
return autosave;
}
public void setOutputFormat(String outputFormat) {
if(outputFormat.equalsIgnoreCase("csv") || outputFormat.equalsIgnoreCase("tsv")) {
beeLine.info("Format " + outputFormat + " is deprecated, please use " + outputFormat + "2");
}
this.outputFormat = outputFormat;
}
public String getOutputFormat() {
return outputFormat;
}
public void setTrimScripts(boolean trimScripts) {
this.trimScripts = trimScripts;
}
public boolean getTrimScripts() {
return trimScripts;
}
public void setMaxHeight(int maxHeight) {
this.maxHeight = maxHeight;
}
public int getMaxHeight() {
return maxHeight;
}
@Ignore
public File getPropertiesFile() {
return rcFile;
}
public Map<String, String> getHiveVariables() {
return hiveVariables;
}
public void setHiveVariables(Map<String, String> hiveVariables) {
this.hiveVariables = hiveVariables;
}
public boolean isAllowMultiLineCommand() {
return allowMultiLineCommand;
}
public void setAllowMultiLineCommand(boolean allowMultiLineCommand) {
this.allowMultiLineCommand = allowMultiLineCommand;
}
/**
* Use getNullString() to get the null string to be used.
* @return true if null representation should be an empty string
*/
public boolean getNullEmptyString() {
return nullEmptyString;
}
public void setNullEmptyString(boolean nullStringEmpty) {
this.nullEmptyString = nullStringEmpty;
}
@Ignore
public String getNullString(){
return nullEmptyString ? "" : DEFAULT_NULL_STRING;
}
public Map<String, String> getHiveConfVariables() {
return hiveConfVariables;
}
public void setHiveConfVariables(Map<String, String> hiveConfVariables) {
this.hiveConfVariables = hiveConfVariables;
}
public boolean getTruncateTable() {
return truncateTable;
}
public void setTruncateTable(boolean truncateTable) {
this.truncateTable = truncateTable;
}
public char getDelimiterForDSV() {
return delimiterForDSV;
}
public void setDelimiterForDSV(char delimiterForDSV) {
this.delimiterForDSV = delimiterForDSV;
}
@Ignore
public HiveConf getConf() {
return conf;
}
public void setHelpAsked(boolean helpAsked) {
this.helpAsked = helpAsked;
}
public boolean isHelpAsked() {
return helpAsked;
}
public void setBeelineSiteUrlsAsked(boolean beelineSiteUrlsAsked) {
this.beelineSiteUrlsAsked = beelineSiteUrlsAsked;
}
public boolean isBeelineSiteUrlsAsked() {
return beelineSiteUrlsAsked;
}
public String getLastConnectedUrl(){
return lastConnectedUrl;
}
public void setLastConnectedUrl(String lastConnectedUrl){
this.lastConnectedUrl = lastConnectedUrl;
}
public String getDelimiter() {
return this.delimiter;
}
public void setDelimiter(String delimiter) {
this.delimiter = delimiter;
}
@Ignore
public static Env getEnv(){
return env;
}
@Ignore
public static void setEnv(Env envToUse){
env = envToUse;
}
}