[
https://issues.apache.org/jira/browse/BEAM-4044?focusedWorklogId=97994&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-97994
]
ASF GitHub Bot logged work on BEAM-4044:
----------------------------------------
Author: ASF GitHub Bot
Created on: 03/May/18 18:16
Start Date: 03/May/18 18:16
Worklog Time Spent: 10m
Work Description: akedin commented on a change in pull request #5224:
[BEAM-4044] [SQL] Add tables via TableStore in Schema, execute DDL in Calcite
model
URL: https://github.com/apache/beam/pull/5224#discussion_r185884947
##########
File path:
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/BeamSqlTableProvider.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.beam.sdk.extensions.sql.meta.provider;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.BeamSqlTable;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+
+/**
+ * A {@code BeamSqlTableProvider} provides read only set of {@code
BeamSqlTable}.
+ */
+public class BeamSqlTableProvider implements TableProvider {
+ private final String typeName;
+ private final Map<String, BeamSqlTable> tables;
+
+ public BeamSqlTableProvider(String typeName, Map<String, BeamSqlTable>
tables) {
+ this.typeName = typeName;
+ this.tables = tables;
+ }
+
+ @Override public String getTableType() {
+ return typeName;
+ }
+
+ @Override
+ public void createTable(Table table) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void dropTable(String tableName) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public List<Table> listTables() {
+ List<Table> list = new ArrayList(tables.size());
+ for (Map.Entry<String, BeamSqlTable> table : tables.entrySet()) {
+ list.add(Table.builder()
+ .type(getTableType())
+ .name(table.getKey())
+ .columns(Collections.emptyList())
+ .build());
Review comment:
nit: I would rewrite it this way:
```java
tables
.values()
.stream()
.map(sqlTable ->
Table
.builder()
.type(getTableType())
.name(sqlTable.getKey())
.columns(Collections.emptyList())
.build())
.collect(toList());
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 97994)
Time Spent: 10h 40m (was: 10.5h)
> Take advantage of Calcite DDL
> -----------------------------
>
> Key: BEAM-4044
> URL: https://issues.apache.org/jira/browse/BEAM-4044
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql
> Reporter: Andrew Pilloud
> Assignee: Andrew Pilloud
> Priority: Major
> Time Spent: 10h 40m
> Remaining Estimate: 0h
>
> In Calcite 1.15 support for abstract DDL moved into calcite core. We should
> take advantage of that.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)