-
Notifications
You must be signed in to change notification settings - Fork 864
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
7 changed files
with
448 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
40 changes: 40 additions & 0 deletions
40
...nals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoContext.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* 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.kyuubi.engine.trino | ||
|
||
import java.util.concurrent.atomic.AtomicReference | ||
|
||
import io.trino.client.ClientSession | ||
import okhttp3.OkHttpClient | ||
|
||
class TrinoContext( | ||
val httpClient: OkHttpClient, | ||
val clientSession: AtomicReference[ClientSession]) { | ||
|
||
def getClientSession: ClientSession = clientSession.get | ||
|
||
def setCurrentSchema(schema: String): Unit = { | ||
clientSession.set(ClientSession.builder(clientSession.get).withSchema(schema).build()) | ||
} | ||
|
||
} | ||
|
||
object TrinoContext { | ||
def apply(httpClient: OkHttpClient, clientSession: ClientSession): TrinoContext = | ||
new TrinoContext(httpClient, new AtomicReference(clientSession)) | ||
} |
195 changes: 195 additions & 0 deletions
195
...ls/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoStatement.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,195 @@ | ||
/* | ||
* 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.kyuubi.engine.trino | ||
|
||
import java.util.ArrayList | ||
import java.util.concurrent.ArrayBlockingQueue | ||
|
||
import scala.collection.JavaConverters._ | ||
import scala.collection.mutable.ArrayBuffer | ||
import scala.concurrent.ExecutionContext.Implicits.global | ||
import scala.concurrent.Future | ||
import scala.concurrent.duration | ||
import scala.concurrent.duration.Duration | ||
import scala.util.control.Breaks._ | ||
|
||
import com.google.common.base.Verify | ||
import io.trino.client.ClientSession | ||
import io.trino.client.Column | ||
import io.trino.client.StatementClient | ||
import io.trino.client.StatementClientFactory | ||
|
||
import org.apache.kyuubi.KyuubiSQLException | ||
import org.apache.kyuubi.engine.trino.TrinoStatement._ | ||
|
||
/** | ||
* Trino client communicate with trino cluster. | ||
*/ | ||
class TrinoStatement(trinoContext: TrinoContext, sql: String) { | ||
|
||
private lazy val trino = StatementClientFactory | ||
.newStatementClient(trinoContext.httpClient, trinoContext.getClientSession, sql) | ||
|
||
def getTrinoClient: StatementClient = trino | ||
|
||
def getCurrentDatabase: String = trinoContext.getClientSession.getSchema | ||
|
||
def getColumns: List[Column] = { | ||
while (trino.isRunning) { | ||
val results = trino.currentStatusInfo() | ||
val columns = results.getColumns() | ||
if (columns != null) { | ||
return columns.asScala.toList | ||
} | ||
trino.advance() | ||
} | ||
Verify.verify(trino.isFinished()) | ||
val finalStatus = trino.finalStatusInfo() | ||
if (finalStatus.getError == null) { | ||
throw KyuubiSQLException(s"Query has no columns (#${finalStatus.getId})") | ||
} else { | ||
throw KyuubiSQLException( | ||
s"Query failed (#${finalStatus.getId}): ${finalStatus.getError.getMessage}") | ||
} | ||
} | ||
|
||
/** | ||
* Execute sql and return ResultSet. | ||
*/ | ||
def execute(): Iterable[List[Any]] = { | ||
val rowQueue = new ArrayBlockingQueue[List[Any]](MAX_QUEUED_ROWS) | ||
|
||
val future = Future[Unit] { | ||
while (trino.isRunning) { | ||
val data = trino.currentData().getData() | ||
if (data != null) { | ||
data.asScala.map(_.asScala.toList) | ||
.foreach(e => putOrThrow(rowQueue, e)) | ||
} | ||
trino.advance() | ||
} | ||
} | ||
future.onComplete { | ||
case _ => putOrThrow(rowQueue, END_TOKEN) | ||
} | ||
|
||
val rowBuffer = new ArrayList[List[Any]](MAX_BUFFERED_ROWS) | ||
var bufferStart = System.nanoTime() | ||
val result = ArrayBuffer[List[Any]]() | ||
try { | ||
breakable { | ||
while (!future.isCompleted) { | ||
val atEnd = drainDetectingEnd(rowQueue, rowBuffer, MAX_BUFFERED_ROWS, END_TOKEN) | ||
if (!atEnd) { | ||
// Flush if needed | ||
if (rowBuffer.size() >= MAX_BUFFERED_ROWS || | ||
Duration.fromNanos(bufferStart).compareTo(MAX_BUFFER_TIME) >= 0) { | ||
result ++= rowBuffer.asScala | ||
rowBuffer.clear() | ||
bufferStart = System.nanoTime() | ||
} | ||
|
||
val row = rowQueue.poll(MAX_BUFFER_TIME.toMillis, duration.MILLISECONDS) | ||
row match { | ||
case END_TOKEN => break | ||
case null => | ||
case _ => rowBuffer.add(row) | ||
} | ||
} | ||
} | ||
} | ||
if (!rowQueue.isEmpty()) { | ||
drainDetectingEnd(rowQueue, rowBuffer, Integer.MAX_VALUE, END_TOKEN) | ||
} | ||
val finalStatus = trino.finalStatusInfo() | ||
if (finalStatus.getError() != null) { | ||
val exception = KyuubiSQLException( | ||
s"Query ${finalStatus.getId} failed: ${finalStatus.getError.getMessage}") | ||
throw exception | ||
} | ||
|
||
updateTrinoContext() | ||
} catch { | ||
case e: Exception => | ||
throw KyuubiSQLException(e) | ||
} | ||
result | ||
} | ||
|
||
def updateTrinoContext(): Unit = { | ||
val session = trinoContext.getClientSession | ||
|
||
var builder = ClientSession.builder(session) | ||
// update catalog and schema | ||
if (trino.getSetCatalog.isPresent || trino.getSetSchema.isPresent) { | ||
builder = builder | ||
.withCatalog(trino.getSetCatalog.orElse(session.getCatalog)) | ||
.withSchema(trino.getSetSchema.orElse(session.getSchema)) | ||
} | ||
|
||
// update path if present | ||
if (trino.getSetPath.isPresent) { | ||
builder = builder.withPath(trino.getSetPath.get) | ||
} | ||
|
||
// update session properties if present | ||
if (!trino.getSetSessionProperties.isEmpty || !trino.getResetSessionProperties.isEmpty) { | ||
val properties = session.getProperties.asScala.clone() | ||
properties ++= trino.getSetSessionProperties.asScala | ||
properties --= trino.getResetSessionProperties.asScala | ||
builder = builder.withProperties(properties.asJava) | ||
} | ||
|
||
trinoContext.clientSession.set(builder.build()) | ||
} | ||
|
||
private def drainDetectingEnd( | ||
rowQueue: ArrayBlockingQueue[List[Any]], | ||
buffer: ArrayList[List[Any]], | ||
maxBufferSize: Int, | ||
endToken: List[Any]): Boolean = { | ||
val drained = rowQueue.drainTo(buffer, maxBufferSize - buffer.size) | ||
if (drained > 0 && buffer.get(buffer.size() - 1) == endToken) { | ||
buffer.remove(buffer.size() - 1); | ||
true | ||
} else { | ||
false | ||
} | ||
} | ||
|
||
private def putOrThrow(rowQueue: ArrayBlockingQueue[List[Any]], e: List[Any]): Unit = { | ||
try { | ||
rowQueue.put(e) | ||
} catch { | ||
case e: InterruptedException => | ||
Thread.currentThread().interrupt() | ||
throw new RuntimeException(e) | ||
} | ||
} | ||
} | ||
|
||
object TrinoStatement { | ||
final private val MAX_QUEUED_ROWS = 50000 | ||
final private val MAX_BUFFERED_ROWS = 10000 | ||
final private val MAX_BUFFER_TIME = Duration(3, duration.SECONDS) | ||
final private val END_TOKEN = List[Any]() | ||
|
||
def apply(trinoContext: TrinoContext, sql: String): TrinoStatement = { | ||
new TrinoStatement(trinoContext, sql) | ||
} | ||
} |
32 changes: 32 additions & 0 deletions
32
...kyuubi-trino-engine/src/test/scala/org/apache/kyuubi/engine/trino/TrinoContextSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,32 @@ | ||
/* | ||
* 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.kyuubi.engine.trino | ||
|
||
class TrinoContextSuite extends WithTrinoLocalServer { | ||
|
||
test("set current schema") { | ||
val trinoContext = TrinoContext(httpClient, session) | ||
|
||
val trinoStatement = TrinoStatement(trinoContext, "select 1") | ||
assert("tiny" === trinoStatement.getCurrentDatabase) | ||
|
||
trinoContext.setCurrentSchema("sf1") | ||
val trinoStatement2 = TrinoStatement(trinoContext, "select 1") | ||
assert("sf1" === trinoStatement2.getCurrentDatabase) | ||
} | ||
} |
63 changes: 63 additions & 0 deletions
63
...uubi-trino-engine/src/test/scala/org/apache/kyuubi/engine/trino/TrinoStatementSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,63 @@ | ||
/* | ||
* 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.kyuubi.engine.trino | ||
|
||
import org.apache.kyuubi.KyuubiSQLException | ||
|
||
class TrinoStatementSuite extends WithTrinoLocalServer { | ||
|
||
test("test query") { | ||
info(connectUrl) | ||
val trinoStatement = TrinoStatement(TrinoContext(httpClient, session), "select 1") | ||
val schema = trinoStatement.getColumns | ||
val resultSet = trinoStatement.execute() | ||
|
||
assert(schema.size === 1) | ||
assert(schema(0).getName === "_col0") | ||
|
||
assert(resultSet.toIterator.hasNext) | ||
assert(resultSet.toIterator.next() === List(1)) | ||
|
||
val trinoStatement2 = TrinoStatement(TrinoContext(httpClient, session), "show schemas") | ||
val schema2 = trinoStatement2.getColumns | ||
val resultSet2 = trinoStatement2.execute() | ||
|
||
assert(schema2.size === 1) | ||
assert(resultSet2.toIterator.hasNext) | ||
} | ||
|
||
test("test update session") { | ||
val trinoStatement = TrinoStatement(TrinoContext(httpClient, session), "select 1") | ||
val schema = trinoStatement.getColumns | ||
|
||
assert(schema.size === 1) | ||
assert(schema(0).getName === "_col0") | ||
assert("tiny" === trinoStatement.getCurrentDatabase) | ||
|
||
val trinoStatement2 = TrinoStatement(TrinoContext(httpClient, session), "use sf1") | ||
trinoStatement2.execute() | ||
|
||
assert("sf1" === trinoStatement2.getCurrentDatabase) | ||
} | ||
|
||
test("test exception") { | ||
val trinoStatement = TrinoStatement(TrinoContext(httpClient, session), "use kyuubi") | ||
val e1 = intercept[KyuubiSQLException](trinoStatement.execute()) | ||
assert(e1.getMessage.contains("Schema does not exist: tpch.kyuubi")) | ||
} | ||
} |
Oops, something went wrong.