11/*
2- * Licensed to the Apache Software Foundation (ASF) under one
3- * or more contributor license agreements. See the NOTICE file
4- * distributed with this work for additional information
5- * regarding copyright ownership. The ASF licenses this file
6- * to you under the Apache License, Version 2.0 (the
7- * "License"); you may not use this file except in compliance
8- * with the License. You may obtain a copy of the License at
9- *
10- * http://www.apache.org/licenses/LICENSE-2.0
11- *
12- * Unless required by applicable law or agreed to in writing,
13- * software distributed under the License is distributed on an
14- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15- * KIND, either express or implied. See the License for the
16- * specific language governing permissions and limitations
17- * under the License.
18- */
2+ * Licensed to the Apache Software Foundation (ASF) under one
3+ * or more contributor license agreements. See the NOTICE file
4+ * distributed with this work for additional information
5+ * regarding copyright ownership. The ASF licenses this file
6+ * to you under the Apache License, Version 2.0 (the
7+ * "License"); you may not use this file except in compliance
8+ * with the License. You may obtain a copy of the License at
9+ *
10+ * http://www.apache.org/licenses/LICENSE-2.0
11+ *
12+ * Unless required by applicable law or agreed to in writing,
13+ * software distributed under the License is distributed on an
14+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+ * KIND, either express or implied. See the License for the
16+ * specific language governing permissions and limitations
17+ * under the License.
18+ */
1919
2020package org .apache .samza .sql .translator ;
2121
3636import org .apache .samza .operators .functions .FilterFunction ;
3737import org .apache .samza .operators .functions .MapFunction ;
3838import org .apache .samza .serializers .NoOpSerde ;
39- import org .apache .samza .sql .SamzaSqlInputTransformer ;
4039import org .apache .samza .sql .SamzaSqlInputMessage ;
40+ import org .apache .samza .sql .SamzaSqlInputTransformer ;
4141import org .apache .samza .sql .data .SamzaSqlRelMessage ;
4242import org .apache .samza .sql .interfaces .SamzaRelConverter ;
4343import org .apache .samza .sql .interfaces .SqlIOConfig ;
4848import org .apache .samza .table .descriptors .CachingTableDescriptor ;
4949import org .apache .samza .table .descriptors .RemoteTableDescriptor ;
5050
51+
5152/**
5253 * Translator to translate the TableScans in relational graph to the corresponding input streams in the StreamGraph
5354 * implementation
@@ -78,7 +79,7 @@ public void init(Context context) {
7879
7980 @ Override
8081 public boolean apply (SamzaSqlInputMessage samzaSqlInputMessage ) {
81- return !relConverter . isSystemMessage ( samzaSqlInputMessage .getKeyAndMessageKV () );
82+ return !samzaSqlInputMessage .getMetadata (). isSystemMessage ( );
8283 }
8384 }
8485
@@ -147,11 +148,11 @@ private void updateMetrics(Instant startProcessing, Instant endProcessing) {
147148 queryInputEvents .inc ();
148149 processingTime .update (Duration .between (startProcessing , endProcessing ).toMillis ());
149150 }
150-
151151 } // ScanMapFunction
152152
153- void translate (final TableScan tableScan , final String queryLogicalId , final String logicalOpId , final TranslatorContext context ,
154- Map <String , DelegatingSystemDescriptor > systemDescriptors , Map <String , MessageStream <SamzaSqlInputMessage >> inputMsgStreams ) {
153+ void translate (final TableScan tableScan , final String queryLogicalId , final String logicalOpId ,
154+ final TranslatorContext context , Map <String , DelegatingSystemDescriptor > systemDescriptors ,
155+ Map <String , MessageStream <SamzaSqlInputMessage >> inputMsgStreams ) {
155156 StreamApplicationDescriptor streamAppDesc = context .getStreamAppDescriptor ();
156157 List <String > tableNameParts = tableScan .getTable ().getQualifiedName ();
157158 String sourceName = SqlIOConfig .getSourceFromSourceParts (tableNameParts );
@@ -162,9 +163,9 @@ void translate(final TableScan tableScan, final String queryLogicalId, final Str
162163 final String streamId = sqlIOConfig .getStreamId ();
163164 final String source = sqlIOConfig .getSource ();
164165
165- final boolean isRemoteTable = sqlIOConfig .getTableDescriptor ().isPresent () &&
166- ( sqlIOConfig .getTableDescriptor ().get () instanceof RemoteTableDescriptor ||
167- sqlIOConfig . getTableDescriptor () .get () instanceof CachingTableDescriptor );
166+ final boolean isRemoteTable = sqlIOConfig .getTableDescriptor ().isPresent () && (
167+ sqlIOConfig .getTableDescriptor ().get () instanceof RemoteTableDescriptor || sqlIOConfig . getTableDescriptor ()
168+ .get () instanceof CachingTableDescriptor );
168169
169170 // For remote table, we don't have an input stream descriptor. The table descriptor is already defined by the
170171 // SqlIOResolverFactory.
@@ -181,22 +182,55 @@ void translate(final TableScan tableScan, final String queryLogicalId, final Str
181182 systemDescriptors .put (systemName , systemDescriptor );
182183 } else {
183184 /* in SamzaSQL, there should be no systemDescriptor setup by user, so this branch happens only
184- * in case of Fan-OUT (i.e., same input stream used in multiple sql statements), or when same input
185- * used twice in same sql statement (e.g., select ... from input as i1, input as i2 ...), o.w., throw error */
185+ * in case of Fan-OUT (i.e., same input stream used in multiple sql statements), or when same input
186+ * used twice in same sql statement (e.g., select ... from input as i1, input as i2 ...), o.w., throw error */
186187 if (systemDescriptor .getTransformer ().isPresent ()) {
187188 InputTransformer existingTransformer = systemDescriptor .getTransformer ().get ();
188189 if (!(existingTransformer instanceof SamzaSqlInputTransformer )) {
189- throw new SamzaException ("SamzaSQL Exception: existing transformer for " + systemName + " is not SamzaSqlInputTransformer" );
190+ throw new SamzaException (
191+ "SamzaSQL Exception: existing transformer for " + systemName + " is not SamzaSqlInputTransformer" );
190192 }
191193 }
192194 }
193195
194196 InputDescriptor inputDescriptor = systemDescriptor .getInputDescriptor (streamId , new NoOpSerde <>());
195- MessageStream <SamzaSqlRelMessage > samzaSqlRelMessageStream =
196- inputMsgStreams .computeIfAbsent (source , v -> streamAppDesc .getInputStream (inputDescriptor ))
197- .filter (new FilterSystemMessageFunction (sourceName , queryId ))
198- .map (new ScanMapFunction (sourceName , queryId , queryLogicalId , logicalOpId ));
197+
198+ if (!inputMsgStreams .containsKey (source )) {
199+ MessageStream <SamzaSqlInputMessage > inputMsgStream = streamAppDesc .getInputStream (inputDescriptor );
200+ inputMsgStreams .put (source , inputMsgStream .map (new SystemMessageMapperFunction (source , queryId )));
201+ }
202+ MessageStream <SamzaSqlRelMessage > samzaSqlRelMessageStream = inputMsgStreams .get (source )
203+ .filter (new FilterSystemMessageFunction (sourceName , queryId ))
204+ .map (new ScanMapFunction (sourceName , queryId , queryLogicalId , logicalOpId ));
199205
200206 context .registerMessageStream (tableScan .getId (), samzaSqlRelMessageStream );
201207 }
208+
209+ /**
210+ * Function that populates whether the message is a system message.
211+ * TODO This should ideally be populated by the InputTransformer in future.
212+ */
213+ private static class SystemMessageMapperFunction implements MapFunction <SamzaSqlInputMessage , SamzaSqlInputMessage > {
214+ private final String source ;
215+ private final int queryId ;
216+ private transient SamzaRelConverter relConverter ;
217+
218+ public SystemMessageMapperFunction (String source , int queryId ) {
219+ this .source = source ;
220+ this .queryId = queryId ;
221+ }
222+
223+ @ Override
224+ public void init (Context context ) {
225+ TranslatorContext translatorContext =
226+ ((SamzaSqlApplicationContext ) context .getApplicationTaskContext ()).getTranslatorContexts ().get (queryId );
227+ relConverter = translatorContext .getMsgConverter (source );
228+ }
229+
230+ @ Override
231+ public SamzaSqlInputMessage apply (SamzaSqlInputMessage message ) {
232+ message .getMetadata ().setIsSystemMessage (relConverter .isSystemMessage (message .getKeyAndMessageKV ()));
233+ return message ;
234+ }
235+ }
202236}
0 commit comments