-
Notifications
You must be signed in to change notification settings - Fork 4.6k
/
BeeLine.java
2469 lines (2131 loc) · 72.5 KB
/
BeeLine.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.
*/
/*
* 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.BufferedReader;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.PrintStream;
import java.io.SequenceInputStream;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.JarURLConnection;
import java.net.URL;
import java.net.URLConnection;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.Driver;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.SQLWarning;
import java.sql.Statement;
import java.text.ChoiceFormat;
import java.text.MessageFormat;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Date;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.Properties;
import java.util.ResourceBundle;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.SortedSet;
import java.util.StringTokenizer;
import java.util.TreeMap;
import java.util.TreeSet;
import java.util.jar.Attributes;
import java.util.jar.Manifest;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.GnuParser;
import org.apache.commons.cli.OptionBuilder;
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.io.IOUtils;
import org.apache.hive.beeline.cli.CliOptionsProcessor;
import org.apache.hive.beeline.hs2connection.BeelineConfFileParseException;
import org.apache.hive.beeline.hs2connection.BeelineSiteParseException;
import org.apache.hive.beeline.hs2connection.BeelineSiteParser;
import org.apache.hive.beeline.hs2connection.HS2ConnectionFileParser;
import org.apache.hive.beeline.hs2connection.HS2ConnectionFileUtils;
import org.apache.hive.beeline.hs2connection.HiveSiteHS2ConnectionFileParser;
import org.apache.hive.beeline.hs2connection.UserHS2ConnectionFileParser;
import org.apache.hive.common.util.ShutdownHookManager;
import org.apache.hive.jdbc.JdbcUriParseException;
import org.apache.hive.jdbc.Utils;
import org.apache.hive.jdbc.Utils.JdbcConnectionParams;
import org.apache.thrift.transport.TTransportException;
import com.google.common.annotations.VisibleForTesting;
import jline.console.ConsoleReader;
import jline.console.completer.Completer;
import jline.console.completer.FileNameCompleter;
import jline.console.completer.StringsCompleter;
import jline.console.history.FileHistory;
/**
* A console SQL shell with command completion.
* <p>
* TODO:
* <ul>
* <li>User-friendly connection prompts</li>
* <li>Page results</li>
* <li>Handle binary data (blob fields)</li>
* <li>Implement command aliases</li>
* <li>Stored procedure execution</li>
* <li>Binding parameters to prepared statements</li>
* <li>Scripting language</li>
* <li>XA transactions</li>
* </ul>
*
*/
@SuppressWarnings("static-access")
public class BeeLine implements Closeable {
private static final ResourceBundle resourceBundle =
ResourceBundle.getBundle(BeeLine.class.getSimpleName());
private final BeeLineSignalHandler signalHandler;
private final Runnable shutdownHook;
private static final String separator = System.getProperty("line.separator");
private boolean exit = false;
private final DatabaseConnections connections = new DatabaseConnections();
public static final String COMMAND_PREFIX = "!";
private Collection<Driver> drivers = null;
private final BeeLineOpts opts = new BeeLineOpts(this, System.getProperties());
private String lastProgress = null;
private final Map<SQLWarning, Date> seenWarnings = new HashMap<SQLWarning, Date>();
private final Commands commands = new Commands(this);
private OutputFile scriptOutputFile = null;
private OutputFile recordOutputFile = null;
private PrintStream outputStream = new PrintStream(System.out, true);
private PrintStream errorStream = new PrintStream(System.err, true);
private InputStream inputStream = System.in;
private ConsoleReader consoleReader;
private List<String> batch = null;
private final Reflector reflector = new Reflector(this);
private String dbName = null;
private String currentDatabase = null;
private FileHistory history;
// Indicates if this instance of beeline is running in compatibility mode, or beeline mode
private boolean isBeeLine = true;
// Indicates that we are in test mode.
// Print only the errors, the operation log and the query results.
private boolean isTestMode = false;
private static final Options options = new Options();
public static final String BEELINE_DEFAULT_JDBC_DRIVER = "org.apache.hive.jdbc.HiveDriver";
public static final String DEFAULT_DATABASE_NAME = "default";
private static final String SCRIPT_OUTPUT_PREFIX = ">>>";
private static final int SCRIPT_OUTPUT_PAD_SIZE = 5;
private static final int ERRNO_OK = 0;
private static final int ERRNO_ARGS = 1;
private static final int ERRNO_OTHER = 2;
private static final String HIVE_VAR_PREFIX = "--hivevar";
private static final String HIVE_CONF_PREFIX = "--hiveconf";
private static final String PROP_FILE_PREFIX = "--property-file";
static final String PASSWD_MASK = "[passwd stripped]";
private final Map<Object, Object> formats = map(new Object[] {
"vertical", new VerticalOutputFormat(this),
"table", new TableOutputFormat(this),
"csv2", new SeparatedValuesOutputFormat(this, ','),
"tsv2", new SeparatedValuesOutputFormat(this, '\t'),
"dsv", new SeparatedValuesOutputFormat(this, BeeLineOpts.DEFAULT_DELIMITER_FOR_DSV),
"csv", new DeprecatedSeparatedValuesOutputFormat(this, ','),
"tsv", new DeprecatedSeparatedValuesOutputFormat(this, '\t'),
"xmlattr", new XMLAttributeOutputFormat(this),
"xmlelements", new XMLElementOutputFormat(this),
});
private List<String> supportedLocalDriver =
new ArrayList<String>(Arrays.asList("com.mysql.jdbc.Driver", "org.postgresql.Driver"));
final CommandHandler[] commandHandlers = new CommandHandler[] {
new ReflectiveCommandHandler(this, new String[] {"quit", "done", "exit"},
null),
new ReflectiveCommandHandler(this, new String[] {"connect", "open"},
new Completer[] {new StringsCompleter(getConnectionURLExamples())}),
new ReflectiveCommandHandler(this, new String[] {"describe"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"indexes"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"primarykeys"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"exportedkeys"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"manual"},
null),
new ReflectiveCommandHandler(this, new String[] {"importedkeys"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"procedures"},
null),
new ReflectiveCommandHandler(this, new String[] {"tables"},
null),
new ReflectiveCommandHandler(this, new String[] {"typeinfo"},
null),
new ReflectiveCommandHandler(this, new String[] {"columns"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"reconnect"},
null),
new ReflectiveCommandHandler(this, new String[] {"dropall"},
new Completer[] {new TableNameCompletor(this)}),
new ReflectiveCommandHandler(this, new String[] {"history"},
null),
new ReflectiveCommandHandler(this, new String[] {"metadata"},
new Completer[] {
new StringsCompleter(getMetadataMethodNames())}),
new ReflectiveCommandHandler(this, new String[] {"nativesql"},
null),
new ReflectiveCommandHandler(this, new String[] {"dbinfo"},
null),
new ReflectiveCommandHandler(this, new String[] {"rehash"},
null),
new ReflectiveCommandHandler(this, new String[] {"verbose"},
null),
new ReflectiveCommandHandler(this, new String[] {"run"},
new Completer[] {new FileNameCompleter()}),
new ReflectiveCommandHandler(this, new String[] {"batch"},
null),
new ReflectiveCommandHandler(this, new String[] {"list"},
null),
new ReflectiveCommandHandler(this, new String[] {"all"},
null),
new ReflectiveCommandHandler(this, new String[] {"go", "#"},
null),
new ReflectiveCommandHandler(this, new String[] {"script"},
new Completer[] {new FileNameCompleter()}),
new ReflectiveCommandHandler(this, new String[] {"record"},
new Completer[] {new FileNameCompleter()}),
new ReflectiveCommandHandler(this, new String[] {"brief"},
null),
new ReflectiveCommandHandler(this, new String[] {"close"},
null),
new ReflectiveCommandHandler(this, new String[] {"closeall"},
null),
new ReflectiveCommandHandler(this, new String[] {"isolation"},
new Completer[] {new StringsCompleter(getIsolationLevels())}),
new ReflectiveCommandHandler(this, new String[] {"outputformat"},
new Completer[] {new StringsCompleter(
formats.keySet().toArray(new String[0]))}),
new ReflectiveCommandHandler(this, new String[] {"autocommit"},
null),
new ReflectiveCommandHandler(this, new String[] {"commit"},
null),
new ReflectiveCommandHandler(this, new String[] {"properties"},
new Completer[] {new FileNameCompleter()}),
new ReflectiveCommandHandler(this, new String[] {"rollback"},
null),
new ReflectiveCommandHandler(this, new String[] {"help", "?"},
null),
new ReflectiveCommandHandler(this, new String[] {"set"},
getOpts().optionCompleters()),
new ReflectiveCommandHandler(this, new String[] {"save"},
null),
new ReflectiveCommandHandler(this, new String[] {"scan"},
null),
new ReflectiveCommandHandler(this, new String[] {"sql"},
null),
new ReflectiveCommandHandler(this, new String[] {"sh"},
null),
new ReflectiveCommandHandler(this, new String[] {"call"},
null),
new ReflectiveCommandHandler(this, new String[] {"nullemptystring"},
new Completer[] {new BooleanCompleter()}),
new ReflectiveCommandHandler(this, new String[]{"addlocaldriverjar"},
null),
new ReflectiveCommandHandler(this, new String[]{"addlocaldrivername"},
null),
new ReflectiveCommandHandler(this, new String[]{"delimiter"},
null)
};
private final Completer beeLineCommandCompleter = new BeeLineCommandCompleter(Arrays.asList(commandHandlers));
static final SortedSet<String> KNOWN_DRIVERS = new TreeSet<String>(Arrays.asList(
new String[] {
"org.apache.hive.jdbc.HiveDriver",
"org.apache.hadoop.hive.jdbc.HiveDriver",
}));
static {
try {
Class.forName("jline.console.ConsoleReader");
} catch (Throwable t) {
throw new ExceptionInInitializerError("jline-missing");
}
}
static {
// -d <driver class>
options.addOption(OptionBuilder
.hasArg()
.withArgName("driver class")
.withDescription("The driver class to use")
.create('d'));
// -u <database url>
options.addOption(OptionBuilder
.hasArg()
.withArgName("database url")
.withDescription("The JDBC URL to connect to")
.create('u'));
// -c <named url in the beeline-hs2-connection.xml>
options.addOption(OptionBuilder
.hasArg()
.withArgName("named JDBC URL in beeline-site.xml")
.withDescription("The named JDBC URL to connect to, which should be present in "
+ "beeline-site.xml as the value of beeline.hs2.jdbc.url.<namedUrl>")
.create('c'));
// -r
options.addOption(OptionBuilder
.withLongOpt("reconnect")
.withDescription("Reconnect to last saved connect url (in conjunction with !save)")
.create('r'));
// -n <username>
options.addOption(OptionBuilder
.hasArg()
.withArgName("username")
.withDescription("The username to connect as")
.create('n'));
// -p <password>
options.addOption(OptionBuilder
.hasArg()
.withArgName("password")
.withDescription("The password to connect as")
.hasOptionalArg()
.create('p'));
// -w (or) --password-file <file>
options.addOption(OptionBuilder
.hasArg()
.withArgName("password-file")
.withDescription("The password file to read password from")
.withLongOpt("password-file")
.create('w'));
// -a <authType>
options.addOption(OptionBuilder
.hasArg()
.withArgName("authType")
.withDescription("The authentication type")
.create('a'));
// -i <init file>
options.addOption(OptionBuilder
.hasArgs()
.withArgName("init")
.withDescription("The script file for initialization")
.create('i'));
// -e <query>
options.addOption(OptionBuilder
.hasArgs()
.withArgName("query")
.withDescription("The query that should be executed")
.create('e'));
// -f <script file>
options.addOption(OptionBuilder
.hasArg()
.withArgName("file")
.withDescription("The script file that should be executed")
.create('f'));
// -help
options.addOption(OptionBuilder
.withLongOpt("help")
.withDescription("Display this message")
.create('h'));
// Substitution option --hivevar
options.addOption(OptionBuilder
.withValueSeparator()
.hasArgs(2)
.withArgName("key=value")
.withLongOpt("hivevar")
.withDescription("Hive variable name and value")
.create());
//hive conf option --hiveconf
options.addOption(OptionBuilder
.withValueSeparator()
.hasArgs(2)
.withArgName("property=value")
.withLongOpt("hiveconf")
.withDescription("Use value for given property")
.create());
// --property-file <file>
options.addOption(OptionBuilder
.hasArg()
.withLongOpt("property-file")
.withDescription("The file to read configuration properties from")
.create());
}
static Manifest getManifest() throws IOException {
URL base = BeeLine.class.getResource("/META-INF/MANIFEST.MF");
URLConnection c = base.openConnection();
if (c instanceof JarURLConnection) {
return ((JarURLConnection) c).getManifest();
}
return null;
}
String getManifestAttribute(String name) {
try {
Manifest m = getManifest();
if (m == null) {
return "??";
}
Attributes attrs = m.getAttributes("beeline");
if (attrs == null) {
return "???";
}
String val = attrs.getValue(name);
if (val == null || "".equals(val)) {
return "????";
}
return val;
} catch (Exception e) {
e.printStackTrace(errorStream);
return "?????";
}
}
String getApplicationTitle() {
Package pack = BeeLine.class.getPackage();
return loc("app-introduction", new Object[] { "Beeline",
pack.getImplementationVersion() == null ? "???" : pack.getImplementationVersion(),
"Apache Hive",
// getManifestAttribute ("Specification-Title"),
// getManifestAttribute ("Implementation-Version"),
// getManifestAttribute ("Implementation-ReleaseDate"),
// getManifestAttribute ("Implementation-Vendor"),
// getManifestAttribute ("Implementation-License"),
});
}
String getApplicationContactInformation() {
return getManifestAttribute("Implementation-Vendor");
}
String loc(String res) {
return loc(res, new Object[0]);
}
String loc(String res, int param) {
try {
return MessageFormat.format(
new ChoiceFormat(resourceBundle.getString(res)).format(param),
new Object[] {new Integer(param)});
} catch (Exception e) {
return res + ": " + param;
}
}
String loc(String res, Object param1) {
return loc(res, new Object[] {param1});
}
String loc(String res, Object param1, Object param2) {
return loc(res, new Object[] {param1, param2});
}
String loc(String res, Object[] params) {
try {
return MessageFormat.format(resourceBundle.getString(res), params);
} catch (Exception e) {
e.printStackTrace(getErrorStream());
try {
return res + ": " + Arrays.asList(params);
} catch (Exception e2) {
return res;
}
}
}
protected String locElapsedTime(long milliseconds) {
if (getOpts().getShowElapsedTime()) {
return loc("time-ms", new Object[] {new Double(milliseconds / 1000d)});
}
return "";
}
/**
* Starts the program.
*/
public static void main(String[] args) throws IOException {
mainWithInputRedirection(args, null);
}
/**
* Starts the program with redirected input. For redirected output,
* setOutputStream() and setErrorStream can be used.
* Exits with 0 on success, 1 on invalid arguments, and 2 on any other error
*
* @param args
* same as main()
*
* @param inputStream
* redirected input, or null to use standard input
*/
public static void mainWithInputRedirection(String[] args, InputStream inputStream)
throws IOException {
BeeLine beeLine = new BeeLine();
try {
int status = beeLine.begin(args, inputStream);
if (!Boolean.getBoolean(BeeLineOpts.PROPERTY_NAME_EXIT)) {
System.exit(status);
}
} finally {
beeLine.close();
}
}
public BeeLine() {
this(true);
}
public BeeLine(boolean isBeeLine) {
this.isBeeLine = isBeeLine;
this.signalHandler = new SunSignalHandler(this);
this.shutdownHook = new Runnable() {
@Override
public void run() {
try {
if (history != null) {
history.setMaxSize(getOpts().getMaxHistoryRows());
history.flush();
}
} catch (IOException e) {
error(e);
} finally {
close();
}
}
};
}
DatabaseConnection getDatabaseConnection() {
return getDatabaseConnections().current();
}
Connection getConnection() throws SQLException {
if (getDatabaseConnections().current() == null
|| getDatabaseConnections().current().getConnection() == null) {
throw new IllegalArgumentException(loc("no-current-connection"));
}
return getDatabaseConnections().current().getConnection();
}
DatabaseMetaData getDatabaseMetaData() {
if (getDatabaseConnections().current() == null) {
throw new IllegalArgumentException(loc("no-current-connection"));
}
if (getDatabaseConnections().current().getDatabaseMetaData() == null) {
throw new IllegalArgumentException(loc("no-current-connection"));
}
return getDatabaseConnections().current().getDatabaseMetaData();
}
public String[] getIsolationLevels() {
return new String[] {
"TRANSACTION_NONE",
"TRANSACTION_READ_COMMITTED",
"TRANSACTION_READ_UNCOMMITTED",
"TRANSACTION_REPEATABLE_READ",
"TRANSACTION_SERIALIZABLE",
};
}
public String[] getMetadataMethodNames() {
try {
TreeSet<String> mnames = new TreeSet<String>();
Method[] m = DatabaseMetaData.class.getDeclaredMethods();
for (int i = 0; m != null && i < m.length; i++) {
mnames.add(m[i].getName());
}
return mnames.toArray(new String[0]);
} catch (Throwable t) {
return new String[0];
}
}
public String[] getConnectionURLExamples() {
return new String[] {
"jdbc:JSQLConnect://<hostname>/database=<database>",
"jdbc:cloudscape:<database>;create=true",
"jdbc:twtds:sqlserver://<hostname>/<database>",
"jdbc:daffodilDB_embedded:<database>;create=true",
"jdbc:datadirect:db2://<hostname>:50000;databaseName=<database>",
"jdbc:inetdae:<hostname>:1433",
"jdbc:datadirect:oracle://<hostname>:1521;SID=<database>;MaxPooledStatements=0",
"jdbc:datadirect:sqlserver://<hostname>:1433;SelectMethod=cursor;DatabaseName=<database>",
"jdbc:datadirect:sybase://<hostname>:5000",
"jdbc:db2://<hostname>/<database>",
"jdbc:hive2://<hostname>",
"jdbc:hsqldb:<database>",
"jdbc:idb:<database>.properties",
"jdbc:informix-sqli://<hostname>:1526/<database>:INFORMIXSERVER=<database>",
"jdbc:interbase://<hostname>//<database>.gdb",
"jdbc:microsoft:sqlserver://<hostname>:1433;DatabaseName=<database>;SelectMethod=cursor",
"jdbc:mysql://<hostname>/<database>?autoReconnect=true",
"jdbc:oracle:thin:@<hostname>:1521:<database>",
"jdbc:pointbase:<database>,database.home=<database>,create=true",
"jdbc:postgresql://<hostname>:5432/<database>",
"jdbc:postgresql:net//<hostname>/<database>",
"jdbc:sybase:Tds:<hostname>:4100/<database>?ServiceName=<database>",
"jdbc:weblogic:mssqlserver4:<database>@<hostname>:1433",
"jdbc:odbc:<database>",
"jdbc:sequelink://<hostname>:4003/[Oracle]",
"jdbc:sequelink://<hostname>:4004/[Informix];Database=<database>",
"jdbc:sequelink://<hostname>:4005/[Sybase];Database=<database>",
"jdbc:sequelink://<hostname>:4006/[SQLServer];Database=<database>",
"jdbc:sequelink://<hostname>:4011/[ODBC MS Access];Database=<database>",
"jdbc:openlink://<hostname>/DSN=SQLServerDB/UID=sa/PWD=",
"jdbc:solid://<hostname>:<port>/<UID>/<PWD>",
"jdbc:dbaw://<hostname>:8889/<database>",
};
}
/**
* Entry point to creating a {@link ColorBuffer} with color
* enabled or disabled depending on the value of {@link BeeLineOpts#getColor}.
*/
ColorBuffer getColorBuffer() {
return new ColorBuffer(getOpts().getColor());
}
/**
* Entry point to creating a {@link ColorBuffer} with color
* enabled or disabled depending on the value of {@link BeeLineOpts#getColor}.
*/
ColorBuffer getColorBuffer(String msg) {
return new ColorBuffer(msg, getOpts().getColor());
}
public class BeelineParser extends GnuParser {
private boolean isPasswordOptionSet = false;
@Override
protected void processOption(String arg, final ListIterator iter) throws ParseException {
if (isBeeLineOpt(arg)) {
processBeeLineOpt(arg);
} else {
//-p with the next argument being for BeeLineOpts
if ("-p".equals(arg)) {
isPasswordOptionSet = true;
if(iter.hasNext()) {
String next = (String) iter.next();
if(isBeeLineOpt(next)) {
processBeeLineOpt(next);
return;
} else {
iter.previous();
}
}
}
super.processOption(arg, iter);
}
}
private void processBeeLineOpt(final String arg) {
String stripped = arg.substring(2, arg.length());
String[] parts = split(stripped, "=");
debug(loc("setting-prop", Arrays.asList(parts)));
if (parts.length >= 2) {
getOpts().set(parts[0], parts[1], true);
} else {
getOpts().set(parts[0], "true", true);
}
}
private boolean isBeeLineOpt(String arg) {
return arg.startsWith("--") && !(HIVE_VAR_PREFIX.equals(arg) || (HIVE_CONF_PREFIX.equals(arg))
|| "--help".equals(arg) || PROP_FILE_PREFIX.equals(arg));
}
}
int initArgsFromCliVars(String[] args) {
List<String> commands = Collections.emptyList();
CliOptionsProcessor optionsProcessor = new CliOptionsProcessor();
if (!optionsProcessor.process(args)) {
return 1;
}
CommandLine commandLine = optionsProcessor.getCommandLine();
Properties confProps = commandLine.getOptionProperties("hiveconf");
for (String propKey : confProps.stringPropertyNames()) {
setHiveConfVar(propKey, confProps.getProperty(propKey));
}
Properties hiveVars = commandLine.getOptionProperties("define");
for (String propKey : hiveVars.stringPropertyNames()) {
getOpts().getHiveConfVariables().put(propKey, hiveVars.getProperty(propKey));
}
Properties hiveVars2 = commandLine.getOptionProperties("hivevar");
for (String propKey : hiveVars2.stringPropertyNames()) {
getOpts().getHiveConfVariables().put(propKey, hiveVars2.getProperty(propKey));
}
getOpts().setScriptFile(commandLine.getOptionValue("f"));
if (commandLine.getOptionValues("i") != null) {
getOpts().setInitFiles(commandLine.getOptionValues("i"));
}
dbName = commandLine.getOptionValue("database");
getOpts().setVerbose(Boolean.parseBoolean(commandLine.getOptionValue("verbose")));
getOpts().setSilent(Boolean.parseBoolean(commandLine.getOptionValue("silent")));
int code = 0;
if (commandLine.getOptionValues("e") != null) {
commands = Arrays.asList(commandLine.getOptionValues("e"));
}
if (!commands.isEmpty() && getOpts().getScriptFile() != null) {
System.err.println("The '-e' and '-f' options cannot be specified simultaneously");
optionsProcessor.printCliUsage();
return 1;
}
if (!commands.isEmpty()) {
embeddedConnect();
connectDBInEmbededMode();
for (Iterator<String> i = commands.iterator(); i.hasNext(); ) {
String command = i.next().toString();
debug(loc("executing-command", command));
if (!dispatch(command)) {
code++;
}
}
exit = true; // execute and exit
}
return code;
}
int initArgs(String[] args) {
List<String> commands = Collections.emptyList();
CommandLine cl;
BeelineParser beelineParser;
try {
beelineParser = new BeelineParser();
cl = beelineParser.parse(options, args);
} catch (ParseException e1) {
output(e1.getMessage());
usage();
return -1;
}
boolean connSuccessful = connectUsingArgs(beelineParser, cl);
// checks if default hs2 connection configuration file is present
// and uses it to connect if found
// no-op if the file is not present
if(!connSuccessful && !exit) {
connSuccessful = defaultBeelineConnect(cl);
}
if (exit) {
return 1;
}
int code = 0;
if (cl.getOptionValues('e') != null) {
commands = Arrays.asList(cl.getOptionValues('e'));
opts.setAllowMultiLineCommand(false); //When using -e, command is always a single line
}
if (!commands.isEmpty() && getOpts().getScriptFile() != null) {
error("The '-e' and '-f' options cannot be specified simultaneously");
return 1;
} else if(!commands.isEmpty() && !connSuccessful) {
error("Cannot run commands specified using -e. No current connection");
return 1;
}
if (!commands.isEmpty()) {
for (Iterator<String> i = commands.iterator(); i.hasNext();) {
String command = i.next().toString();
debug(loc("executing-command", command));
if (!dispatch(command)) {
code++;
}
}
exit = true; // execute and exit
}
return code;
}
/*
* Connects using the command line arguments. There are two
* possible ways to connect here 1. using the cmd line arguments like -u
* or using !properties <property-file>
*/
private boolean connectUsingArgs(BeelineParser beelineParser, CommandLine cl) {
String driver = null, user = null, pass = "", url = null;
String auth = null;
if (cl.hasOption("help")) {
usage();
getOpts().setHelpAsked(true);
return true;
}
Properties hiveVars = cl.getOptionProperties("hivevar");
for (String key : hiveVars.stringPropertyNames()) {
getOpts().getHiveVariables().put(key, hiveVars.getProperty(key));
}
Properties hiveConfs = cl.getOptionProperties("hiveconf");
for (String key : hiveConfs.stringPropertyNames()) {
setHiveConfVar(key, hiveConfs.getProperty(key));
}
driver = cl.getOptionValue("d");
auth = cl.getOptionValue("a");
user = cl.getOptionValue("n");
getOpts().setAuthType(auth);
if (cl.hasOption("w")) {
pass = obtainPasswordFromFile(cl.getOptionValue("w"));
} else {
if (beelineParser.isPasswordOptionSet) {
pass = cl.getOptionValue("p");
}
}
url = cl.getOptionValue("u");
if ((url == null) && cl.hasOption("reconnect")){
// If url was not specified with -u, but -r was present, use that.
url = getOpts().getLastConnectedUrl();
}
getOpts().setInitFiles(cl.getOptionValues("i"));
getOpts().setScriptFile(cl.getOptionValue("f"));
if (url != null) {
// Specifying username/password/driver explicitly will override the values from the url;
// make sure we don't override the values present in the url with empty values.
if (user == null) {
user = Utils.parsePropertyFromUrl(url, JdbcConnectionParams.AUTH_USER);
}
if (pass == null) {
pass = Utils.parsePropertyFromUrl(url, JdbcConnectionParams.AUTH_PASSWD);
}
if (driver == null) {
driver = Utils.parsePropertyFromUrl(url, JdbcConnectionParams.PROPERTY_DRIVER);
}
String com;
String comForDebug;
if(pass != null) {
com = constructCmd(url, user, pass, driver, false);
comForDebug = constructCmd(url, user, pass, driver, true);
} else {
com = constructCmdUrl(url, user, driver, false);
comForDebug = constructCmdUrl(url, user, driver, true);
}
debug(comForDebug);
if (!dispatch(com)) {
exit = true;
return false;
}
return true;
}
// load property file
String propertyFile = cl.getOptionValue("property-file");
if (propertyFile != null) {
try {
this.consoleReader = new ConsoleReader();
} catch (IOException e) {
handleException(e);
}
if (!dispatch("!properties " + propertyFile)) {
exit = true;
return false;
}
}
return false;
}
private void setHiveConfVar(String key, String val) {
getOpts().getHiveConfVariables().put(key, val);
if (HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname.equals(key) && "mr".equals(val)) {
info(HiveConf.generateMrDeprecationWarning());
}
}
private String constructCmd(String url, String user, String pass, String driver, boolean stripPasswd) {
return new StringBuilder()
.append("!connect ")
.append(url)
.append(" ")
.append(user == null || user.length() == 0 ? "''" : user)
.append(" ")
.append(stripPasswd ? PASSWD_MASK : (pass.length() == 0 ? "''" : pass))
.append(" ")
.append((driver == null ? "" : driver))
.toString();
}
/**
* This is an internal method used to create !connect command when -p option is used without
* providing the password on the command line. Connect command returned should be ; separated
* key-value pairs along with the url. We cannot use space separated !connect url user [password]
* [driver] here since both password and driver are optional and there would be no way to
* distinguish if the last string is password or driver
*
* @param url connection url passed using -u argument on the command line
* @param user username passed through command line
* @param driver driver passed through command line -d option
* @param stripPasswd when set to true generates a !connect command which strips the password for
* logging purposes
* @return !connect command
*/
private String constructCmdUrl(String url, String user, String driver,
boolean stripPasswd) {
StringBuilder command = new StringBuilder("!connect ");
command.append(url);
//if the url does not have a database name add the trailing '/'
if(isTrailingSlashNeeded(url)) {
command.append('/');
}
command.append(';');
// if the username is not already available in the URL add the one provided
if (Utils.parsePropertyFromUrl(url, JdbcConnectionParams.AUTH_USER) == null) {
command.append(JdbcConnectionParams.AUTH_USER);
command.append('=');
command.append((user == null || user.length() == 0 ? "''" : user));
}
if (stripPasswd) {
// if password is available in url it needs to be striped
int startIndex = command.indexOf(JdbcConnectionParams.AUTH_PASSWD + "=")
+ JdbcConnectionParams.AUTH_PASSWD.length() + 2;
if(startIndex != -1) {
int endIndex = command.toString().indexOf(";", startIndex);
command.replace(startIndex, (endIndex == -1 ? command.length() : endIndex),
BeeLine.PASSWD_MASK);
}
}
// if the driver is not already available in the URL add the one provided
if (Utils.parsePropertyFromUrl(url, JdbcConnectionParams.PROPERTY_DRIVER) == null
&& driver != null) {
command.append(';');
command.append(JdbcConnectionParams.PROPERTY_DRIVER);
command.append("=");
command.append(driver);
}
return command.toString();
}
/*
* Returns true if trailing slash is needed to be appended to the url
*/
private boolean isTrailingSlashNeeded(String url) {
if (url.toLowerCase().startsWith("jdbc:hive2://")) {
return url.indexOf('/', "jdbc:hive2://".length()) < 0;