Skip to content

Commit 08af4aa

Browse files
committed
Support command to load a csv file
Signed-off-by: Lantao Jin <[email protected]>
1 parent 7a4c206 commit 08af4aa

20 files changed

+549
-13
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,22 @@
1+
FlightNum,Origin,FlightDelay,DistanceMiles,FlightTimeMin,OriginWeather,dayOfWeek,AvgTicketPrice,Carrier,FlightDelayMin,OriginRegion,FlightDelayType,DestAirportID,Dest,FlightTimeHour,Cancelled,DistanceKilometers,OriginCityName,DestWeather,OriginCountry,DestCountry,DestRegion,OriginAirportID,DestCityName,timestamp
2+
RGXY9H5,Chubu Centrair International Airport,false,1619.970725161303,124.1471507959044,Heavy Fog,0,626.1297405910661,OpenSearch Dashboards Airlines,0,SE-BD,No Delay,CAN,Guangzhou Baiyun International Airport,2.06911917993174,true,2607.0901667139924,Tokoname,Clear,JP,CN,SE-BD,NGO,Guangzhou,2019-12-23 11:19:32
3+
WOPNZEP,Munich Airport,true,198.57903689856937,34.9738738474057,Sunny,0,681.9911763989377,OpenSearch Dashboards Airlines,15,DE-BY,Carrier Delay,VE05,Venice Marco Polo Airport,0.5828978974567617,false,319.58198155849124,Munich,Cloudy,DE,IT,IT-34,MUC,Venice,2019-12-23 12:32:26
4+
G9J5O2V,Frankfurt am Main Airport,false,4857.154739888458,651.402736475921,Clear,0,868.0507463122127,OpenSearch Dashboards Airlines,0,DE-HE,No Delay,XIY,Xi'an Xianyang International Airport,10.856712274598683,false,7816.832837711051,Frankfurt am Main,Thunder & Lightning,DE,CN,SE-BD,FRA,Xi'an,2019-12-23 03:48:33
5+
HM80A5V,Itami Airport,false,5862.6666599206,555.0027890084269,Heavy Fog,0,765.0413127727119,Logstash Airways,0,SE-BD,No Delay,TV01,Treviso-Sant'Angelo Airport,9.250046483473783,true,9435.047413143258,Osaka,Clear,JP,IT,IT-34,ITM,Treviso,2019-12-23 19:50:48
6+
84B0Y32,Charles de Gaulle International Airport,false,4397.926660603864,372.51457282541395,Thunder & Lightning,0,913.1638984616233,OpenSearch Dashboards Airlines,0,FR-J,No Delay,STL,St Louis Lambert International Airport,6.208576213756899,false,7077.776883682865,Paris,Thunder & Lightning,FR,US,US-MO,CDG,St Louis,2019-12-23 11:30:48
7+
2AZWPJX,Rajiv Gandhi International Airport,true,0,180,Sunny,0,103.25307304704197,Logstash Airways,180,SE-BD,Security Delay,HYD,Rajiv Gandhi International Airport,3,false,0,Hyderabad,Hail,IN,IN,SE-BD,HYD,Hyderabad,2019-12-23 19:52:54
8+
SFLRI9O,Erie International Tom Ridge Field,false,6961.655654280931,622.4277087379495,Clear,0,775.1109173747694,OpenSearch Dashboards Airlines,0,US-PA,No Delay,CJU,Jeju International Airport,10.373795145632492,false,11203.698757283091,Erie,Clear,US,KR,SE-BD,ERI,Jeju City,2019-12-23 07:32:32
9+
QDQMOD6,Brisbane International Airport,false,8013.330880747018,716.4558873858294,Thunder & Lightning,0,832.082612870741,OpenSearch-Air,0,SE-BD,No Delay,DEN,Denver International Airport,11.94093145643049,false,12896.20597294493,Brisbane,Cloudy,AU,US,US-CO,BNE,Denver,2019-12-23 10:59:26
10+
XTGFN9A,Jorge Chavez International Airport,false,3946.924514217792,396.99745533808243,Thunder & Lightning,0,348.23579123315324,OpenSearch Dashboards Airlines,0,SE-BD,No Delay,YOW,Ottawa Macdonald-Cartier International Airport,6.616624255634707,false,6351.959285409319,Lima,Rain,PE,CA,CA-ON,LIM,Ottawa,2019-12-23 21:10:09
11+
USRQ3KK,Stockholm-Arlanda Airport,false,996.8381561540818,94.36797091633146,Clear,0,661.3465606549407,OpenSearch-Air,0,SE-AB,No Delay,TV01,Treviso-Sant'Angelo Airport,1.572799515272191,false,1604.2555055776347,Stockholm,Clear,SE,IT,IT-34,ARN,Treviso,2019-12-23 04:33:56
12+
PK46NHH,Milano Linate Airport,false,5261.396351845886,604.8140464617903,Rain,0,600.4401843290168,JetBeats,0,IT-25,No Delay,GEG,Spokane International Airport,10.080234107696505,false,8467.396650465065,Milan,Clear,IT,US,US-WA,MI11,Spokane,2019-12-23 20:35:25
13+
G80VHCJ,Bari Karol Wojty__a Airport,false,5630.111629019724,604.0524246328747,Sunny,0,738.260189539631,Logstash Airways,0,IT-75,No Delay,CJU,Jeju International Airport,10.067540410547911,false,9060.78636949312,Bari,Rain,IT,KR,SE-BD,BA02,Jeju City,2019-12-23 10:59:56
14+
PDS4U17,El Dorado International Airport,false,5591.079567130033,499.887241937962,Thunder & Lightning,0,437.9253204442997,OpenSearch-Air,0,SE-BD,No Delay,TO11,Turin Airport,8.331454032299368,false,8997.970354883317,Bogota,Hail,CO,IT,IT-21,BOG,Torino,2019-12-23 10:33:53
15+
2MXRGRK,Abu Dhabi International Airport,false,8160.7690090650885,656.6742320062424,Cloudy,0,825.9174161826418,JetBeats,0,SE-BD,No Delay,ABQ,Albuquerque International Sunport Airport,10.944570533437373,false,13133.484640124847,Abu Dhabi,Thunder & Lightning,AE,US,US-NM,AUH,Albuquerque,2019-12-23 19:27:11
16+
57CZEDA,London Heathrow Airport,true,4757.876231054233,720.4152685405732,Damaging Wind,0,836.1010286937247,OpenSearch-Air,270,GB-ENG,Carrier Delay,XHBU,Ukrainka Air Base,12.006921142342886,false,7657.059565189745,London,Sunny,GB,RU,RU-AMU,LHR,Belogorsk,2019-12-23 18:48:49
17+
5FYALP0,Malpensa International Airport,false,5812.230334559898,492.30936923905085,Damaging Wind,0,417.34744554513884,JetBeats,0,IT-25,No Delay,LAS,McCarran International Airport,8.20515615398418,false,9353.878015541966,Milan,Clear,IT,US,US-NV,MI12,Las Vegas,2019-12-23 10:37:54
18+
HVWAL6J,Comodoro Arturo Merino Benitez International Airport,false,7292.7292896018525,617.7110592550002,Cloudy,0,946.888426456834,Logstash Airways,0,SE-BD,No Delay,PA03,Falcone Borsellino Airport,10.29518432091667,false,11736.510125845005,Santiago,Cloudy,CL,IT,IT-82,SCL,Palermo,2019-12-23 03:54:12
19+
7ORM12S,Leonardo da Vinci___Fiumicino Airport,false,160.39074208529965,23.46580713004768,Sunny,0,118.37483602607261,OpenSearch Dashboards Airlines,0,IT-62,No Delay,PI05,Pisa International Airport,0.39109678550079463,false,258.1238784305245,Rome,Sunny,IT,IT,IT-52,RM11,Pisa,2019-12-23 03:54:12
20+
2P36OEP,New Chitose Airport,true,5340.290617241973,941.1970552595557,Cloudy,0,705.7149863531135,OpenSearch Dashboards Airlines,225,SE-BD,Late Aircraft Delay,VIE,Vienna International Airport,15.686617587659262,false,8594.364663114668,Chitose / Tomakomai,Rain,JP,AT,AT-9,CTS,Vienna,2019-12-23 09:41:52
21+
HLNZHCX,Verona Villafranca Airport,false,0,0,Sunny,0,172.3790782673846,OpenSearch-Air,0,IT-34,No Delay,VR10,Verona Villafranca Airport,0,false,0,Verona,Sunny,IT,IT,IT-34,VR10,Verona,2019-12-23 19:34:51
22+
HLNNULL,Verona Villafranca Airport,,,0,,0,172.3790782673846,OpenSearch-Air,0,IT-34,No Delay,VR10,Verona Villafranca Airport,0,false,0,Verona,Sunny,IT,IT,IT-34,VR10,Verona,
615 Bytes
Binary file not shown.
Binary file not shown.
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,234 @@
1+
/*
2+
* Copyright OpenSearch Contributors
3+
* SPDX-License-Identifier: Apache-2.0
4+
*/
5+
6+
package org.opensearch.flint.spark.ppl
7+
8+
import org.apache.spark.sql.{QueryTest, Row}
9+
import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedStar}
10+
import org.apache.spark.sql.catalyst.expressions.{Alias, And, EqualTo, GreaterThan, Literal}
11+
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project}
12+
import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
13+
import org.apache.spark.sql.streaming.StreamTest
14+
import org.apache.spark.sql.types.{ArrayType, BooleanType, DoubleType, IntegerType, StringType, StructType, TimestampType}
15+
16+
class FlintSparkPPLInputITSuite
17+
extends QueryTest
18+
with LogicalPlanTestUtils
19+
with FlintPPLSuite
20+
with StreamTest {
21+
22+
override def beforeAll(): Unit = {
23+
super.beforeAll()
24+
}
25+
26+
protected override def afterEach(): Unit = {
27+
super.afterEach()
28+
// Stop all streaming jobs if any
29+
spark.streams.active.foreach { job =>
30+
job.stop()
31+
job.awaitTermination()
32+
}
33+
}
34+
35+
test("test csv file source relation") {
36+
val frame = sql(s"""
37+
| input "integ-test/src/integration/resources/opensearch_dashboards_sample_data_flights.csv"
38+
| | where Cancelled = false AND DistanceMiles > 0
39+
| | fields FlightNum, DistanceMiles, FlightTimeMin, OriginWeather, AvgTicketPrice, Carrier, FlightDelayType, timestamp
40+
| | stats avg(AvgTicketPrice) as avg_price_by_weather by OriginWeather
41+
| """.stripMargin)
42+
43+
val results: Array[Row] = frame.collect()
44+
// results.foreach(println(_))
45+
val expectedResults: Array[Row] = Array(
46+
Row(768.1694081139743, "Clear"),
47+
Row(826.1736096641965, "Cloudy"),
48+
Row(600.4401843290168, "Rain"),
49+
Row(512.8754006548804, "Sunny"),
50+
Row(632.8519057524543, "Thunder & Lightning"),
51+
Row(626.7242371194318, "Damaging Wind"))
52+
implicit val oneColRowOrdering: Ordering[Row] = Ordering.by[Row, Double](_.getAs[Double](0))
53+
assert(results.sorted.sameElements(expectedResults.sorted))
54+
55+
val logicalPlan: LogicalPlan = frame.queryExecution.logical
56+
57+
val userSpecifiedSchema = new StructType()
58+
.add("FlightNum", StringType)
59+
.add("Origin", StringType)
60+
.add("FlightDelay", BooleanType)
61+
.add("DistanceMiles", DoubleType)
62+
.add("FlightTimeMin", DoubleType)
63+
.add("OriginWeather", StringType)
64+
.add("dayOfWeek", IntegerType)
65+
.add("AvgTicketPrice", DoubleType)
66+
.add("Carrier", StringType)
67+
.add("FlightDelayMin", IntegerType)
68+
.add("OriginRegion", StringType)
69+
.add("FlightDelayType", StringType)
70+
.add("DestAirportID", StringType)
71+
.add("Dest", StringType)
72+
.add("FlightTimeHour", DoubleType)
73+
.add("Cancelled", BooleanType)
74+
.add("DistanceKilometers", DoubleType)
75+
.add("OriginCityName", StringType)
76+
.add("DestWeather", StringType)
77+
.add("OriginCountry", StringType)
78+
.add("DestCountry", StringType)
79+
.add("DestRegion", StringType)
80+
.add("OriginAirportID", StringType)
81+
.add("DestCityName", StringType)
82+
.add("timestamp", TimestampType)
83+
val dataSource =
84+
DataSource(
85+
spark,
86+
userSpecifiedSchema = Some(userSpecifiedSchema),
87+
className = "csv",
88+
options = Map.empty)
89+
val relation = LogicalRelation(dataSource.resolveRelation(true), isStreaming = false)
90+
val filterExpr = And(
91+
EqualTo(UnresolvedAttribute("Cancelled"), Literal(false)),
92+
GreaterThan(UnresolvedAttribute("DistanceMiles"), Literal(0)))
93+
val filter = Filter(filterExpr, relation)
94+
val project = Project(
95+
Seq(
96+
UnresolvedAttribute("FlightNum"),
97+
UnresolvedAttribute("DistanceMiles"),
98+
UnresolvedAttribute("FlightTimeMin"),
99+
UnresolvedAttribute("OriginWeather"),
100+
UnresolvedAttribute("AvgTicketPrice"),
101+
UnresolvedAttribute("Carrier"),
102+
UnresolvedAttribute("FlightDelayType"),
103+
UnresolvedAttribute("timestamp")),
104+
filter)
105+
val weatherAlias = Alias(UnresolvedAttribute("OriginWeather"), "OriginWeather")()
106+
val groupByAttributes = Seq(weatherAlias)
107+
val aggregateExpressions =
108+
Alias(
109+
UnresolvedFunction(
110+
Seq("AVG"),
111+
Seq(UnresolvedAttribute("AvgTicketPrice")),
112+
isDistinct = false),
113+
"avg_price_by_weather")()
114+
val aggregatePlan =
115+
Aggregate(groupByAttributes, Seq(aggregateExpressions, weatherAlias), project)
116+
val expectedPlan = Project(Seq(UnresolvedStar(None)), aggregatePlan)
117+
assert(compareByString(expectedPlan) === compareByString(logicalPlan))
118+
}
119+
120+
test("test csv file source relation with compression codec") {
121+
val frame = sql(s"""
122+
| input "integ-test/src/integration/resources/opensearch_dashboards_sample_data_flights.csv.gz"
123+
| | where Cancelled = false AND DistanceMiles > 0
124+
| | fields FlightNum, DistanceMiles, FlightTimeMin, OriginWeather, AvgTicketPrice, Carrier, FlightDelayType, timestamp
125+
| | stats avg(AvgTicketPrice) as avg_price_by_weather by OriginWeather
126+
| """.stripMargin)
127+
128+
val results: Array[Row] = frame.collect()
129+
// results.foreach(println(_))
130+
val expectedResults: Array[Row] = Array(
131+
Row(768.1694081139743, "Clear"),
132+
Row(826.1736096641965, "Cloudy"),
133+
Row(600.4401843290168, "Rain"),
134+
Row(512.8754006548804, "Sunny"),
135+
Row(632.8519057524543, "Thunder & Lightning"),
136+
Row(626.7242371194318, "Damaging Wind"))
137+
implicit val oneColRowOrdering: Ordering[Row] = Ordering.by[Row, Double](_.getAs[Double](0))
138+
assert(results.sorted.sameElements(expectedResults.sorted))
139+
140+
val logicalPlan: LogicalPlan = frame.queryExecution.logical
141+
142+
val userSpecifiedSchema = new StructType()
143+
.add("FlightNum", StringType)
144+
.add("Origin", StringType)
145+
.add("FlightDelay", BooleanType)
146+
.add("DistanceMiles", DoubleType)
147+
.add("FlightTimeMin", DoubleType)
148+
.add("OriginWeather", StringType)
149+
.add("dayOfWeek", IntegerType)
150+
.add("AvgTicketPrice", DoubleType)
151+
.add("Carrier", StringType)
152+
.add("FlightDelayMin", IntegerType)
153+
.add("OriginRegion", StringType)
154+
.add("FlightDelayType", StringType)
155+
.add("DestAirportID", StringType)
156+
.add("Dest", StringType)
157+
.add("FlightTimeHour", DoubleType)
158+
.add("Cancelled", BooleanType)
159+
.add("DistanceKilometers", DoubleType)
160+
.add("OriginCityName", StringType)
161+
.add("DestWeather", StringType)
162+
.add("OriginCountry", StringType)
163+
.add("DestCountry", StringType)
164+
.add("DestRegion", StringType)
165+
.add("OriginAirportID", StringType)
166+
.add("DestCityName", StringType)
167+
.add("timestamp", TimestampType)
168+
val dataSource =
169+
DataSource(
170+
spark,
171+
userSpecifiedSchema = Some(userSpecifiedSchema),
172+
className = "csv",
173+
options = Map.empty)
174+
val relation = LogicalRelation(dataSource.resolveRelation(true), isStreaming = false)
175+
val filterExpr = And(
176+
EqualTo(UnresolvedAttribute("Cancelled"), Literal(false)),
177+
GreaterThan(UnresolvedAttribute("DistanceMiles"), Literal(0)))
178+
val filter = Filter(filterExpr, relation)
179+
val project = Project(
180+
Seq(
181+
UnresolvedAttribute("FlightNum"),
182+
UnresolvedAttribute("DistanceMiles"),
183+
UnresolvedAttribute("FlightTimeMin"),
184+
UnresolvedAttribute("OriginWeather"),
185+
UnresolvedAttribute("AvgTicketPrice"),
186+
UnresolvedAttribute("Carrier"),
187+
UnresolvedAttribute("FlightDelayType"),
188+
UnresolvedAttribute("timestamp")),
189+
filter)
190+
val weatherAlias = Alias(UnresolvedAttribute("OriginWeather"), "OriginWeather")()
191+
val groupByAttributes = Seq(weatherAlias)
192+
val aggregateExpressions =
193+
Alias(
194+
UnresolvedFunction(
195+
Seq("AVG"),
196+
Seq(UnresolvedAttribute("AvgTicketPrice")),
197+
isDistinct = false),
198+
"avg_price_by_weather")()
199+
val aggregatePlan =
200+
Aggregate(groupByAttributes, Seq(aggregateExpressions, weatherAlias), project)
201+
val expectedPlan = Project(Seq(UnresolvedStar(None)), aggregatePlan)
202+
assert(compareByString(expectedPlan) === compareByString(logicalPlan))
203+
}
204+
205+
test("test parquet file source relation") {
206+
val frame = sql(s"""
207+
| input "integ-test/src/integration/resources/users.parquet"
208+
| | fields name, favorite_color
209+
| """.stripMargin)
210+
211+
val results: Array[Row] = frame.collect()
212+
// results.foreach(println(_))
213+
val expectedResults: Array[Row] = Array(Row("Alyssa", null), Row("Ben", "red"))
214+
implicit val oneColRowOrdering: Ordering[Row] = Ordering.by[Row, String](_.getAs[String](0))
215+
assert(results.sorted.sameElements(expectedResults.sorted))
216+
217+
val logicalPlan: LogicalPlan = frame.queryExecution.logical
218+
219+
val userSpecifiedSchema = new StructType()
220+
.add("name", StringType)
221+
.add("favorite_color", StringType)
222+
.add("favorite_numbers", new ArrayType(IntegerType, true))
223+
val dataSource =
224+
DataSource(
225+
spark,
226+
userSpecifiedSchema = Some(userSpecifiedSchema),
227+
className = "parquet",
228+
options = Map.empty)
229+
val relation = LogicalRelation(dataSource.resolveRelation(true), isStreaming = false)
230+
val expectedPlan =
231+
Project(Seq(UnresolvedAttribute("name"), UnresolvedAttribute("favorite_color")), relation)
232+
assert(compareByString(expectedPlan) === compareByString(logicalPlan))
233+
}
234+
}

ppl-spark-integration/README.md

+6
Original file line numberDiff line numberDiff line change
@@ -236,6 +236,12 @@ See the next samples of PPL queries :
236236
- `source = catalog.table | where struct_col2.field1.subfield > 'valueA' | sort int_col | fields int_col, struct_col.field1.subfield, struct_col2.field1.subfield`
237237
- `source = catalog.schema.table | where struct_col2.field1.subfield > 'valueA' | sort int_col | fields int_col, struct_col.field1.subfield, struct_col2.field1.subfield`
238238

239+
**Input**
240+
Input command is used to load file as a lookup dataset
241+
- `input "s3://my_bucket/path/csv" | fields a,b,c`
242+
- `input "s3://my_bucket/path/csv.gz" | fields a,b,c`
243+
- `input "s3a://my_bucket/path/parquet" | fields a,b,c`
244+
239245
**Filters**
240246
- `source = table | where a = 1 | fields a,b,c`
241247
- `source = table | where a >= 1 | fields a,b,c`

ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4

+1
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ channels { WHITESPACE, ERRORCHANNEL }
1212
// COMMAND KEYWORDS
1313
SEARCH: 'SEARCH';
1414
DESCRIBE: 'DESCRIBE';
15+
INPUT: 'INPUT';
1516
SHOW: 'SHOW';
1617
FROM: 'FROM';
1718
WHERE: 'WHERE';

ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4

+10
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@ queryStatement
2828
pplCommands
2929
: searchCommand
3030
| describeCommand
31+
| inputCommand
3132
;
3233

3334
commands
@@ -56,6 +57,14 @@ describeCommand
5657
: DESCRIBE tableSourceClause
5758
;
5859

60+
inputCommand
61+
: INPUT urlList
62+
;
63+
64+
urlList
65+
: stringLiteral (COMMA stringLiteral)*
66+
;
67+
5968
showDataSourcesCommand
6069
: SHOW DATASOURCES
6170
;
@@ -830,6 +839,7 @@ keywordsCanBeId
830839
// commands
831840
| SEARCH
832841
| DESCRIBE
842+
| INPUT
833843
| SHOW
834844
| FROM
835845
| WHERE

ppl-spark-integration/src/main/java/org/opensearch/sql/ast/AbstractNodeVisitor.java

+6
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import org.opensearch.sql.ast.tree.Kmeans;
4545
import org.opensearch.sql.ast.tree.Limit;
4646
import org.opensearch.sql.ast.tree.Parse;
47+
import org.opensearch.sql.ast.tree.FileSourceRelation;
4748
import org.opensearch.sql.ast.tree.Project;
4849
import org.opensearch.sql.ast.tree.RareTopN;
4950
import org.opensearch.sql.ast.tree.Relation;
@@ -215,10 +216,15 @@ public T visitHead(Head node, C context) {
215216
public T visitRareTopN(RareTopN node, C context) {
216217
return visitChildren(node, context);
217218
}
219+
218220
public T visitValues(Values node, C context) {
219221
return visitChildren(node, context);
220222
}
221223

224+
public T visitFileSourceRelation(FileSourceRelation node, C context) {
225+
return visitChildren(node, context);
226+
}
227+
222228
public T visitAlias(Alias node, C context) {
223229
return visitChildren(node, context);
224230
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,56 @@
1+
/*
2+
* Copyright OpenSearch Contributors
3+
* SPDX-License-Identifier: Apache-2.0
4+
*/
5+
6+
package org.opensearch.sql.ast.tree;
7+
8+
import com.google.common.collect.ImmutableList;
9+
import org.opensearch.sql.ast.AbstractNodeVisitor;
10+
import org.opensearch.sql.ast.Node;
11+
12+
import java.util.List; /**
13+
* AST node class for a sequence of files (such as CSV files) to read.
14+
*/
15+
public class FileSourceRelation extends UnresolvedPlan {
16+
private final List<String> paths;
17+
private final String format;
18+
private final String compressionCodeName;
19+
20+
public FileSourceRelation(List<String> paths, String format) {
21+
this(paths, format, null);
22+
}
23+
24+
public FileSourceRelation(List<String> paths, String format, String compressionCodeName) {
25+
this.paths = paths;
26+
this.format = format;
27+
this.compressionCodeName = compressionCodeName;
28+
}
29+
30+
public List<String> getPaths() {
31+
return paths;
32+
}
33+
34+
public String getFormat() {
35+
return format;
36+
}
37+
38+
public String getCompressionCodeName() {
39+
return compressionCodeName;
40+
}
41+
42+
@Override
43+
public UnresolvedPlan attach(UnresolvedPlan child) {
44+
throw new UnsupportedOperationException("PathsReader node is supposed to have no child node");
45+
}
46+
47+
@Override
48+
public <T, C> T accept(AbstractNodeVisitor<T, C> nodeVisitor, C context) {
49+
return nodeVisitor.visitFileSourceRelation(this, context);
50+
}
51+
52+
@Override
53+
public List<? extends Node> getChild() {
54+
return ImmutableList.of();
55+
}
56+
}

0 commit comments

Comments
 (0)