Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fork beeline module from Apache Hive 3.1.3 #6109

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions .rat-excludes
Original file line number Diff line number Diff line change
Expand Up @@ -51,3 +51,5 @@ build/scala-*/**
**/node_modules/**
**/gen/*
**/*.tokens
**/BeeLine.properties
**/sql-keywords.properties
3 changes: 0 additions & 3 deletions NOTICE-binary
Original file line number Diff line number Diff line change
Expand Up @@ -189,9 +189,6 @@ JUnit (4.12)

* License: Eclipse Public License

Hive Beeline
Copyright 2022 The Apache Software Foundation

Hive Common
Copyright 2022 The Apache Software Foundation

Expand Down
37 changes: 25 additions & 12 deletions kyuubi-hive-beeline/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -46,18 +46,6 @@
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-beeline</artifactId>
<version>${hive.version}</version>
<exclusions>
<exclusion>
<groupId>*</groupId>
<artifactId>*</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.hive</groupId>
<artifactId>hive-common</artifactId>
Expand Down Expand Up @@ -229,6 +217,31 @@
<skipTests>${skipTests}</skipTests>
</configuration>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<executions>
<execution>
<id>copy</id>
<goals>
<goal>copy</goal>
</goals>
<phase>process-test-resources</phase>
<configuration>
<artifactItems>
<artifactItem>
<groupId>org.postgresql</groupId>
<artifactId>postgresql</artifactId>
<version>${postgresql.version}</version>
<overWrite>true</overWrite>
<outputDirectory>${project.build.directory}</outputDirectory>
</artifactItem>
</artifactItems>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
<outputDirectory>target/classes</outputDirectory>
<testOutputDirectory>target/test-classes</testOutputDirectory>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/*
* 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.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import jline.console.completer.Completer;
import jline.console.completer.NullCompleter;

/** An abstract implementation of CommandHandler. */
public abstract class AbstractCommandHandler implements CommandHandler {
private final BeeLine beeLine;
private final String name;
private final String[] names;
private final String helpText;
private Completer[] parameterCompleters = new Completer[0];

protected transient Throwable lastException;

public AbstractCommandHandler(
BeeLine beeLine, String[] names, String helpText, Completer[] completors) {
this.beeLine = beeLine;
name = names[0];
this.names = names;
this.helpText = helpText;
if (completors == null || completors.length == 0) {
parameterCompleters = new Completer[] {new NullCompleter()};
} else {
List<Completer> c = new LinkedList<Completer>(Arrays.asList(completors));
c.add(new NullCompleter());
parameterCompleters = c.toArray(new Completer[0]);
}
}

@Override
public String getHelpText() {
return helpText;
}

@Override
public String getName() {
return name;
}

@Override
public String[] getNames() {
return names;
}

@Override
public String matches(String line) {
if (line == null || line.length() == 0) {
return null;
}

String[] parts = beeLine.split(line);
if (parts == null || parts.length == 0) {
return null;
}

for (String name2 : names) {
if (name2.startsWith(parts[0])) {
return name2;
}
}
return null;
}

public void setParameterCompleters(Completer[] parameterCompleters) {
this.parameterCompleters = parameterCompleters;
}

@Override
public Completer[] getParameterCompleters() {
return parameterCompleters;
}

@Override
public Throwable getLastException() {
return lastException;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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;

/** Abstract OutputFormat. */
abstract class AbstractOutputFormat implements OutputFormat {

public int print(Rows rows) {
int count = 0;
Rows.Row header = (Rows.Row) rows.next();
printHeader(header);

while (rows.hasNext()) {
printRow(rows, header, (Rows.Row) rows.next());
count++;
}
printFooter(header);
return count;
}

abstract void printHeader(Rows.Row header);

abstract void printFooter(Rows.Row header);

abstract void printRow(Rows rows, Rows.Row header, Rows.Row row);
}
Loading
Loading