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

Acknowledge reception of data in TrinoResult #220

Merged
merged 4 commits into from
Sep 22, 2022
Merged
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 .github/workflows/ci.yml
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,8 @@ jobs:
include:
# Test with older Trino versions for backward compatibility
- { python: "3.10", trino: "351" } # first Trino version
# Test with Trino version that requires result set to be fully exhausted
- { python: "3.10", trino: "395" }
env:
TRINO_VERSION: "${{ matrix.trino }}"
steps:
Expand Down
13 changes: 6 additions & 7 deletions tests/integration/test_dbapi_integration.py
Original file line number Diff line number Diff line change
Expand Up @@ -153,8 +153,8 @@ def test_execute_many_without_params(trino_connection):
cur = trino_connection.cursor()
cur.execute("CREATE TABLE memory.default.test_execute_many_without_param (value varchar)")
cur.fetchall()
cur.executemany("INSERT INTO memory.default.test_execute_many_without_param (value) VALUES (?)", [])
with pytest.raises(TrinoUserError) as e:
cur.executemany("INSERT INTO memory.default.test_execute_many_without_param (value) VALUES (?)", [])
cur.fetchall()
assert "Incorrect number of parameters: expected 1 but found 0" in str(e.value)

Expand Down Expand Up @@ -883,13 +883,12 @@ def test_transaction_autocommit(trino_connection_in_autocommit):
with trino_connection_in_autocommit as connection:
connection.start_transaction()
cur = connection.cursor()
cur.execute(
"""
CREATE TABLE memory.default.nation
AS SELECT * from tpch.tiny.nation
""")

with pytest.raises(TrinoUserError) as transaction_error:
cur.execute(
"""
CREATE TABLE memory.default.nation
AS SELECT * from tpch.tiny.nation
""")
cur.fetchall()
assert "Catalog only supports writes using autocommit: memory" \
in str(transaction_error.value)
Expand Down
4 changes: 2 additions & 2 deletions tests/unit/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -37,10 +37,10 @@ def sample_post_response_data():
"""

yield {
"nextUri": "coordinator:8080/v1/statement/20210817_140827_00000_arvdv/1",
"nextUri": "https://coordinator:8080/v1/statement/20210817_140827_00000_arvdv/1",
"id": "20210817_140827_00000_arvdv",
"taskDownloadUris": [],
"infoUri": "http://coordinator:8080/query.html?20210817_140827_00000_arvdv",
"infoUri": "https://coordinator:8080/query.html?20210817_140827_00000_arvdv",
"stats": {
"scheduled": False,
"runningSplits": 0,
Expand Down
4 changes: 1 addition & 3 deletions tests/unit/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -892,9 +892,7 @@ def test_trino_result_response_headers():
'X-Trino-Fake-2': 'two',
})

result = TrinoResult(
query=mock_trino_query,
)
result = TrinoResult(query=mock_trino_query, rows=[])
assert result.response_headers == mock_trino_query.response_headers


Expand Down
34 changes: 28 additions & 6 deletions tests/unit/test_dbapi.py
Original file line number Diff line number Diff line change
Expand Up @@ -51,21 +51,28 @@ def test_http_session_is_defaulted_when_not_specified(mock_client):


@httprettified
def test_token_retrieved_once_per_auth_instance(sample_post_response_data):
def test_token_retrieved_once_per_auth_instance(sample_post_response_data, sample_get_response_data):
token = str(uuid.uuid4())
challenge_id = str(uuid.uuid4())

redirect_server = f"{REDIRECT_RESOURCE}/{challenge_id}"
token_server = f"{TOKEN_RESOURCE}/{challenge_id}"

post_statement_callback = PostStatementCallback(redirect_server, token_server, [token], sample_post_response_data)
get_statement_callback = PostStatementCallback(redirect_server, token_server, [token], sample_get_response_data)

# bind post statement
# bind post statement to submit query
httpretty.register_uri(
method=httpretty.POST,
uri=f"{SERVER_ADDRESS}:8080{constants.URL_STATEMENT_PATH}",
body=post_statement_callback)

# bind get statement for result retrieval
httpretty.register_uri(
method=httpretty.GET,
uri=f"{SERVER_ADDRESS}:8080{constants.URL_STATEMENT_PATH}/20210817_140827_00000_arvdv/1",
body=get_statement_callback)

# bind get token
get_token_callback = GetTokenCallback(token_server, token)
httpretty.register_uri(
Expand Down Expand Up @@ -108,21 +115,29 @@ def test_token_retrieved_once_per_auth_instance(sample_post_response_data):


@httprettified
def test_token_retrieved_once_when_authentication_instance_is_shared(sample_post_response_data):
def test_token_retrieved_once_when_authentication_instance_is_shared(sample_post_response_data,
sample_get_response_data):
token = str(uuid.uuid4())
challenge_id = str(uuid.uuid4())

redirect_server = f"{REDIRECT_RESOURCE}/{challenge_id}"
token_server = f"{TOKEN_RESOURCE}/{challenge_id}"

post_statement_callback = PostStatementCallback(redirect_server, token_server, [token], sample_post_response_data)
get_statement_callback = PostStatementCallback(redirect_server, token_server, [token], sample_get_response_data)

# bind post statement
# bind post statement to submit query
httpretty.register_uri(
method=httpretty.POST,
uri=f"{SERVER_ADDRESS}:8080{constants.URL_STATEMENT_PATH}",
body=post_statement_callback)

# bind get statement for result retrieval
httpretty.register_uri(
method=httpretty.GET,
uri=f"{SERVER_ADDRESS}:8080{constants.URL_STATEMENT_PATH}/20210817_140827_00000_arvdv/1",
body=get_statement_callback)

# bind get token
get_token_callback = GetTokenCallback(token_server, token)
httpretty.register_uri(
Expand Down Expand Up @@ -166,21 +181,28 @@ def test_token_retrieved_once_when_authentication_instance_is_shared(sample_post


@httprettified
def test_token_retrieved_once_when_multithreaded(sample_post_response_data):
def test_token_retrieved_once_when_multithreaded(sample_post_response_data, sample_get_response_data):
token = str(uuid.uuid4())
challenge_id = str(uuid.uuid4())

redirect_server = f"{REDIRECT_RESOURCE}/{challenge_id}"
token_server = f"{TOKEN_RESOURCE}/{challenge_id}"

post_statement_callback = PostStatementCallback(redirect_server, token_server, [token], sample_post_response_data)
get_statement_callback = PostStatementCallback(redirect_server, token_server, [token], sample_get_response_data)

# bind post statement
# bind post statement to submit query
httpretty.register_uri(
method=httpretty.POST,
uri=f"{SERVER_ADDRESS}:8080{constants.URL_STATEMENT_PATH}",
body=post_statement_callback)

# bind get statement for result retrieval
httpretty.register_uri(
method=httpretty.GET,
uri=f"{SERVER_ADDRESS}:8080{constants.URL_STATEMENT_PATH}/20210817_140827_00000_arvdv/1",
body=get_statement_callback)

# bind get token
get_token_callback = GetTokenCallback(token_server, token)
httpretty.register_uri(
Expand Down
42 changes: 25 additions & 17 deletions trino/client.py
Original file line number Diff line number Diff line change
Expand Up @@ -592,30 +592,36 @@ class TrinoResult(object):
https://docs.python.org/3/library/stdtypes.html#generator-types
"""

def __init__(self, query, rows=None):
def __init__(self, query, rows: List[Any]):
self._query = query
self._rows = rows or []
# Initial rows from the first POST request
mdesmet marked this conversation as resolved.
Show resolved Hide resolved
self._rows = rows
self._rownumber = 0

@property
def rows(self):
return self._rows

@rows.setter
def rows(self, rows):
self._rows = rows

@property
def rownumber(self) -> int:
return self._rownumber

def __iter__(self):
# Initial fetch from the first POST request
for row in self._rows:
self._rownumber += 1
yield row
self._rows = None

# Subsequent fetches from GET requests until next_uri is empty.
while not self._query.finished:
rows = self._query.fetch()
for row in rows:
# A query only transitions to a FINISHED state when the results are fully consumed:
# The reception of the data is acknowledged by calling the next_uri before exposing the data through dbapi.
while not self._query.finished or self._rows is not None:
next_rows = self._query.fetch() if not self._query.finished else None
for row in self._rows:
self._rownumber += 1
logger.debug("row %s", row)
yield row

self._rows = next_rows

@property
def response_headers(self):
return self._query.response_headers
Expand All @@ -641,7 +647,7 @@ def __init__(
self._request = request
self._update_type = None
self._sql = sql
self._result = TrinoResult(self)
self._result: Optional[TrinoResult] = None
self._response_headers = None
self._experimental_python_types = experimental_python_types
self._row_mapper: Optional[RowMapper] = None
Expand All @@ -652,7 +658,7 @@ def columns(self):
while not self._columns and not self.finished and not self.cancelled:
# Columns are not returned immediately after query is submitted.
# Continue fetching data until columns information is available and push fetched rows into buffer.
self._result._rows += self.fetch()
self._result.rows += self.fetch()
return self._columns

@property
Expand Down Expand Up @@ -697,8 +703,11 @@ def execute(self, additional_http_headers=None) -> TrinoResult:
self._finished = True

rows = self._row_mapper.map(status.rows) if self._row_mapper else status.rows

self._result = TrinoResult(self, rows)

# Execute should block until at least one row is received
while not self.finished and not self.cancelled and len(self._result.rows) == 0:
self._result.rows += self.fetch()
return self._result

def _update_state(self, status):
Expand Down Expand Up @@ -921,8 +930,7 @@ class RowMapper:
"""
Maps a row of data given a list of mapping functions
"""

def __init__(self, columns=[]):
def __init__(self, columns):
self.columns = columns

def map(self, rows):
Expand Down
2 changes: 1 addition & 1 deletion trino/dbapi.py
Original file line number Diff line number Diff line change
Expand Up @@ -322,7 +322,7 @@ def _prepare_statement(self, operation, statement_name):
operation=operation
)

# Send prepare statement. Copy the _request object to avoid poluting the
# Send prepare statement. Copy the _request object to avoid polluting the
# one that is going to be used to execute the actual operation.
query = trino.client.TrinoQuery(copy.deepcopy(self._request), sql=sql,
experimental_python_types=self._experimental_pyton_types)
Expand Down
6 changes: 3 additions & 3 deletions trino/sqlalchemy/dialect.py
Original file line number Diff line number Diff line change
Expand Up @@ -231,7 +231,7 @@ def get_view_definition(self, connection: Connection, view_name: str, schema: st
"""
).strip()
res = connection.execute(sql.text(query), schema=schema, view=view_name)
return res.scalar()
return res.scalar_one_or_none()
hashhar marked this conversation as resolved.
Show resolved Hide resolved

def get_indexes(self, connection: Connection, table_name: str, schema: str = None, **kw) -> List[Dict[str, Any]]:
if not self.has_table(connection, table_name, schema):
Expand Down Expand Up @@ -284,7 +284,7 @@ def get_table_comment(self, connection: Connection, table_name: str, schema: str
sql.text(query),
catalog_name=catalog_name, schema_name=schema_name, table_name=table_name
)
return dict(text=res.scalar())
return dict(text=res.scalar_one_or_none())
except error.TrinoQueryError as e:
if e.error_name in (
error.PERMISSION_DENIED,
Expand Down Expand Up @@ -326,7 +326,7 @@ def _get_server_version_info(self, connection: Connection) -> Any:
query = "SELECT version()"
try:
res = connection.execute(sql.text(query))
version = res.scalar()
version = res.scalar_one()
return tuple([version])
except exc.ProgrammingError as e:
logger.debug(f"Failed to get server version: {e.orig.message}")
Expand Down