Open njanakiev opened 3 years ago
For anyone wanting to work on this, this would probably be similar to how we handle PostgreSQL's json and jsonb types
GeoAlchemy2 has wrapper classes for parsing and handling them (eg. WKBElement)
i have the same erro too. can i have some way to read postgis geometry by trino?
For a workaround until this is implemented see https://trinodb.slack.com/archives/CFLB9AMBN/p1624597418210800?thread_ts=1624544478.194000&cid=CFLB9AMBN
As far as I understand PostGIS stores geometries in the WKB format including more metadata like SRID:
It's true.
And If data SRID
is not necessary, should we implement this by simply adding ST_AsBinary around Geometry
type column when it's reading into memory?
I achieved this by overriding buildSql
method in PostgreSqlClient.java
:
@Override
public PreparedStatement buildSql(ConnectorSession session, Connection connection, JdbcSplit split, JdbcTableHandle table, List<JdbcColumnHandle> columns)
throws SQLException
{
Map<String, String> supposedColumnExpressions = new HashMap<>();
for (JdbcColumnHandle column : columns) {
JdbcTypeHandle jdbcTypeHandle = column.getJdbcTypeHandle();
if (jdbcTypeHandle.getJdbcTypeName().isPresent() && jdbcTypeHandle.getJdbcTypeName().get().equals("geometry")) {
String columnName = column.getColumnName();
log.info("Find geometry type, changing '%s' to '%s'", columnName, "ST_AsBinary(\"" + columnName + "\")");
supposedColumnExpressions.put(columnName, "ST_AsBinary(\"" + columnName + "\")");
}
}
ImmutableMap.Builder<String, String> builder = ImmutableMap.builder();
for (Map.Entry<String, String> entry : supposedColumnExpressions.entrySet()) {
builder.put(entry.getKey(), entry.getValue());
}
Map<String, String> columnExpressions = builder.build();
PreparedQuery preparedQuery = prepareQuery(session, connection, table, Optional.empty(), columns, columnExpressions, Optional.of(split));
return new QueryBuilder(this).prepareStatement(session, connection, preparedQuery);
}
To see the difference, I have a simple query: SELECT * FROM postgis.public.points;
with unsupported-type-handling=CONVERT_TO_VARCHAR
on in catalog properties.
This is the output before changing:
trino> SELECT * FROM postgis.public.points;
fid | name | geom
-----+-----------+--------------------------------------------
1 | big mom | 0101000000000000000000F03F000000000000F03F
2 | small dad | 0101000000000000000000F03F000000000000F03F
(2 rows)
This is the output after changing:
trino> SELECT * FROM postgis.public.points;
fid | name | geom
-----+-----------+----------------------------------------------
1 | big mom | \x0101000000000000000000f03f000000000000f03f
2 | small dad | \x0101000000000000000000f03f000000000000f03f
(2 rows)
The result of geom
has a small change which I think this illustrates ST_AsBinary
function works.
Now I'm trying to link the type to Trino's Geometry
. But I failed to get 'Geometry' Type by typeManager, check: https://github.com/trinodb/trino/issues/9845.
If you have any suggestions, feel free to comment.
Hi, some more progress here(keep up with the following changes).
Currently I try more to implement Geometry
reading by:
Adding ColumnMapping
function in PostgreSqlClient.java
(need to import geospatial plugin)
private static ColumnMapping geometryColumnMapping()
{
return ColumnMapping.sliceMapping(
GEOMETRY,
(resultSet, columnIndex) -> stGeomFromBinary(wrappedBuffer(resultSet.getBytes(columnIndex))),
(statement, index, value) -> { throw new TrinoException(NOT_SUPPORTED, "Geometry type is not supported for INSERT"); },
DISABLE_PUSHDOWN);
}
And adding lines in toColumnMapping
function in PostgreSqlClient.java
switch (jdbcTypeName) {
case "geometry":
return Optional.of(geometryColumnMapping());
case "money":
return Optional.of(moneyColumnMapping());
Finally I got result with a little trick like:
trino> SELECT geom FROM postg.public.points;
geom
-------------
POINT (1 1)
POINT (1 1)
(2 rows)
Query 20211109_024831_00006_d6k7e, FINISHED, 1 node
Splits: 17 total, 17 done (100.00%)
0.25 [2 rows, 0B] [8 rows/s, 0B/s]
The reason for my trick is related with the type verifying process, specifically related with lines in JdbcRecordCursor
in jdbc plugin:
verify(
columnHandle.getColumnType().equals(columnMapping.getType()),
"Type mismatch: column handle has type %s but %s is mapped to %s",
columnHandle.getColumnType(), columnHandle.getJdbcTypeHandle(), columnMapping.getType());
I can't pass this verification until I override GeometryType
's equal
method.
If I don't do that I will be caught with error: Type mismatch: column handle has type Geometry but JdbcTypeHandle{..} is mapped to Geometry
I guess there is some problem in GeometryType
for it's never used in a `columnHandle, I'll update clues in #9845 to follow up with the above problem.
There is also a dialogue in slack channel: https://app.slack.com/client/TFKPX15NC/CGB0QHWSW/thread/CGB0QHWSW-1636012275.260300
@findepi @hashhar Sorry for bothering, but I have been working on the following problem for days and didn't get significant progress. I think with this problem solved we'll be very close to the final solution.
I closed #9845. I found getting GEOMETRY
type by typeManager is the right usage:
this.jsonType = typeManager.getType(new TypeSignature(JSON));
Now I'm really close to the final solution. One more thing to pass through is to pass postgres-connector's test.
GEOMETRY
type haven't been register to testing environment.
After this problem is solved I'll consider creating a PR for this issue. After merge, we'll have WKB read ability to PostGIS Geometry Columns.
@njanakiev, @lerenah I've created a PR for this issue. But I suppose it won't be merged very quickly. You can try the connector in my branch.
@njanakiev, @lerenah I've created a PR for this issue. But I suppose it won't be merged very quickly. You can try the connector in my branch.
Thanks, will give it a shot!
any luck the geometry
pr will be merged soon? we really want to try it out with our postgis databases :)
any luck the
geometry
pr will be merged soon? we really want to try it out with our postgis databases :)
If possible I will give some time to retry solving this issue. The last PR is holding for too long time, I my self don't think it could be successfully merged.
any luck the
geometry
pr will be merged soon? we really want to try it out with our postgis databases :)If possible I will give some time to retry solving this issue. The last PR is holding for too long time, I my self don't think it could be successfully merged.
we patched your pr and played a bit. yes it shows geometry better, we could do some cross db sjoin now :) btw, it's there any spatial function on where/join push down implemented? it seems scanning the whole table
btw, it's there any spatial function on where/join push down implemented? it seems scanning the whole table
No, currently these pushdowns should be implemented by yourself. Filter expression can be easily implemented in JDBC connectors if you understand the usage of expression rewriter in #7994. But these functionality could be very limited and risky currently.
I start focusing on these issues this time last year, but moved to another work this year. And since I can see your company in your profile. I guess you are working on similar thing like last year I do. To support better spatial query ability, Trino definitely has a long way to go, so you should be very careful to think about the cost if you decide to use Trino.
Hello, thanks for the great project!
While trying to access geometry columns with the PostgreSQL connector, I wasn't able to access the PostGIS geometry column inside a table:
The same query works with Python together Pandas and SQLAlchemy:
As far as I understand PostGIS stores geometries in the WKB format including more metadata like SRID:
I think this would be a great addition to do spatial joins with federated queries by simply treating the geometry columns as WKB without the projection for example. PostGIS differentiates between Geometries and Geography. Geometry columns are more common from what I know.
Tested on Presto version 344 (with Java 11)