diff --git a/dev/.rat-excludes b/dev/.rat-excludes index e12dc994b0842..dd1783fb0e697 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -118,3 +118,4 @@ announce.tmpl vote.tmpl SessionManager.java SessionHandler.java +.*blacklist diff --git a/examples/pom.xml b/examples/pom.xml index a099f1e042e99..5e4838df09712 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -51,48 +51,72 @@ org.apache.spark spark-core_${scala.binary.version} ${project.version} - provided + compile org.apache.spark spark-streaming_${scala.binary.version} ${project.version} - provided + compile org.apache.spark spark-mllib_${scala.binary.version} ${project.version} - provided + compile org.apache.spark spark-hive_${scala.binary.version} ${project.version} - provided + compile org.apache.spark spark-graphx_${scala.binary.version} ${project.version} - provided + compile org.apache.spark spark-streaming-kafka-0-10_${scala.binary.version} ${project.version} - provided + compile + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + compile + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + compile org.apache.spark spark-sql-kafka-0-10_${scala.binary.version} ${project.version} - provided + compile + + + org.apache.spark + spark-graph-api_${scala.binary.version} + ${project.version} + compile + + + org.apache.spark + spark-cypher_${scala.binary.version} + ${project.version} + compile org.apache.commons commons-math3 - provided + compile org.scalacheck @@ -102,17 +126,142 @@ org.scala-lang scala-library - provided + compile - com.github.scopt - scopt_${scala.binary.version} - 3.7.1 + log4j + log4j + compile + + + org.apache.hadoop + hadoop-client + compile + + + org.apache.hadoop + hadoop-hdfs + ${hadoop.version} + + + org.slf4j + slf4j-api + compile + + + org.slf4j + slf4j-log4j12 + compile + + + org.eclipse.jetty + jetty-http + compile + + + org.eclipse.jetty + jetty-continuation + compile + + + org.eclipse.jetty + jetty-servlet + compile + + + org.eclipse.jetty + jetty-servlets + compile + + + org.eclipse.jetty + jetty-proxy + compile + + + org.eclipse.jetty + jetty-client + compile + + + org.eclipse.jetty + jetty-util + compile + + + org.eclipse.jetty + jetty-security + compile + + + org.eclipse.jetty + jetty-plus + compile + + + org.eclipse.jetty + jetty-server + compile + + + com.google.guava + guava + compile + + + org.jpmml + pmml-model + compile + + + org.apache.parquet + parquet-column + compile + + + org.apache.parquet + parquet-hadoop + compile + + + org.apache.parquet + parquet-avro + compile ${hive.parquet.group} parquet-hadoop-bundle - provided + compile + + + com.fasterxml.jackson.core + jackson-core + compile + + + com.fasterxml.jackson.core + jackson-databind + compile + + + com.fasterxml.jackson.core + jackson-annotations + compile + + + org.codehaus.jackson + jackson-core-asl + compile + + + org.codehaus.jackson + jackson-mapper-asl + compile + + + com.github.scopt + scopt_${scala.binary.version} + 3.7.0 @@ -151,7 +300,7 @@ org.apache.spark spark-streaming-kinesis-asl_${scala.binary.version} ${project.version} - provided + compile diff --git a/examples/src/main/resources/movies/acted_in.csv b/examples/src/main/resources/movies/acted_in.csv new file mode 100644 index 0000000000000..f03abbcd5ff43 --- /dev/null +++ b/examples/src/main/resources/movies/acted_in.csv @@ -0,0 +1,173 @@ +source,target,id,roles +23,15,33,[Lt. Sam Weinberg] +4,105,138,"[Bill Smoke,Haskell Moore,Tadeusz Kesselring,Nurse Noakes,Boardman Mephi,Old Georgie]" +146,162,239,[Bob Hinson] +82,81,102,[Harry Burns] +41,37,53,[Bob Sugar] +34,67,85,[Kathleen Kelly] +53,150,214,[Kelly Frears] +160,159,229,[Joanne Herring] +19,137,193,[Jack Brennan] +25,15,35,[Pfc. Louden Downey] +28,15,37,[Man in Bar] +71,73,91,[Sam Baldwin] +16,15,26,[Lt. Daniel Kaffee] +133,130,187,"[""Wild Bill"" Wharton]" +142,141,198,"[Robert ""Bobby"" Ciaro]" +142,152,217,[Martini] +34,29,45,[Carole] +71,67,84,[Joe Fox] +123,121,163,[Pops] +38,37,50,[Dorothy Boyd] +71,144,202,[Jim Lovell] +1,10,15,[Neo] +1,11,22,[Kevin Lomax] +58,62,81,[Nels Gudmundsson] +134,130,188,[Burt Hammersmith] +57,56,75,[Annie Collins-Nielsen] +135,130,189,[Melinda Moores] +17,154,219,[Harry Sanborn] +71,78,98,[Joe Banks] +2,0,1,[Trinity] +19,15,29,[Capt. Jack Ross] +120,121,168,[Cass Jones] +129,128,175,[Mika Coretti] +4,9,11,[Agent Smith] +17,141,197,[Hoffa] +134,144,206,[Ken Mattingly] +1,154,221,[Julian Mercer] +75,73,94,[Walter] +71,150,213,[Chuck Noland] +12,85,112,[Tina] +60,56,73,[Chris Nielsen] +89,87,116,[Jimmy McGinty] +18,15,28,[Lt. Cdr. JoAnne Galloway] +34,81,103,[Sally Albright] +30,29,41,[Charlie] +71,161,232,"[Hero Boy,Father,Conductor,Hobo,Scrooge,Santa Claus]" +131,130,183,[John Coffey] +22,56,74,[Albert Lewis] +17,152,216,[Randle McMurphy] +120,116,155,[Dascomb] +71,105,137,"[Zachry,Dr. Henry Goose,Isaac Sachs,Dermot Hoggins]" +70,92,122,[Duane] +64,62,80,[Kazuo Miyamoto] +140,157,226,[Rupert Burns] +65,62,82,[Judge Fielding] +2,10,16,[Trinity] +42,37,54,[Laurel Boyd] +47,46,61,[Chris Chambers] +22,37,49,[Rod Tidwell] +118,116,153,[Eric Finch] +3,10,17,[Morpheus] +51,92,119,[Admiral] +139,137,192,[David Frost] +133,137,195,"[James Reston, Jr.]" +103,100,135,[J-Bone] +65,130,186,[Warden Hal Moores] +60,157,225,[Andrew Marin] +155,154,220,[Erica Barry] +119,116,154,[High Chancellor Adam Sutler] +86,85,111,[Faye Dolan] +145,144,204,[Gene Kranz] +165,162,236,[Kit Keller] +99,97,129,[Bill Munny] +89,95,126,[Sen. Kevin Keeley] +68,67,87,[Patricia Eden] +138,137,191,[Richard Nixon] +84,81,105,[Jess] +69,67,88,[Kevin Jackson] +34,78,99,"[DeDe,Angelica Graynamore,Patricia Graynamore]" +117,116,152,[Evey Hammond] +146,144,205,[Fred Haise] +16,37,48,[Jerry Maguire] +112,111,147,[Sir Leight Teabing] +39,37,51,[Avery Bishop] +4,10,18,[Agent Smith] +20,15,30,[Lt. Jonathan Kendrick] +49,46,60,[Gordie Lachance] +53,52,69,[Carol Connelly] +80,78,100,[Baw] +102,100,134,[Jane] +76,73,95,[Greg] +114,111,149,[Silas] +94,92,121,[Squad Leader] +1,100,132,[Johnny Mnemonic] +101,100,133,[Takahashi] +3,0,2,[Morpheus] +70,67,89,[George Pappas] +90,87,117,[Clifford Franklin] +71,111,146,[Dr. Robert Langdon] +63,62,79,[Ishmael Chambers] +43,37,55,[Marcee Tidwell] +80,95,125,[Albert Goldman] +163,162,238,"[""All the Way"" Mae Mordabito]" +40,46,62,[Vern Tessio] +71,130,182,[Paul Edgecomb] +164,162,235,[Dottie Hinson] +126,121,166,[Trixie] +124,121,164,[Mom] +94,147,210,[Eddie] +59,56,77,[The Face] +71,162,234,[Jimmy Dugan] +71,85,110,[Mr. White] +20,46,65,[Ace Merrill] +89,97,130,[Little Bill Daggett] +48,46,63,[Teddy Duchamp] +98,97,128,[English Bob] +146,147,208,[Bill Harding] +113,111,148,[Sophie Neveu] +1,9,8,[Neo] +1,87,114,[Shane Falco] +2,9,9,[Trinity] +127,128,174,[Raizo] +44,37,56,[Ray Boyd] +24,15,34,[Lt. Col. Matthew Andrew Markinson] +88,87,115,[Annabelle Farrell] +31,29,42,[Iceman] +33,29,44,[Viper] +4,0,3,[Agent Smith] +32,29,43,[Goose] +77,73,96,[Becky] +24,141,199,[Frank Fitzsimmons] +125,121,165,[Racer X] +143,141,200,"[Peter ""Pete"" Connelly]" +122,121,162,[Speed Racer] +77,162,237,[Doris Murphy] +12,11,23,[Mary Ann Lomax] +26,15,36,[Dr. Stone] +19,144,203,[Jack Swigert] +17,52,68,[Melvin Udall] +83,81,104,[Marie] +148,159,230,[Gust Avrakotos] +60,95,124,[Armand Goldman] +22,52,71,[Frank Sachs] +132,130,184,"[Brutus ""Brutal"" Howell]" +71,159,228,[Rep. Charlie Wilson] +34,73,92,[Annie Reed] +54,67,86,[Frank Navasky] +107,105,140,"[Vyvyan Ayrs,Captain Molyneux,Timothy Cavendish]" +106,105,139,"[Luisa Rey,Jocasta Ayrs,Ovid,Meronym]" +1,0,0,[Neo] +54,52,70,[Simon Bishop] +120,128,177,[Ryan Maslow] +50,46,64,[Denny Lachance] +21,15,31,[Cpl. Jeffrey Barnes] +53,147,209,[Dr. Jo Harding] +148,147,211,"[Dustin ""Dusty"" Davis]" +64,128,176,[Takeshi] +51,46,66,[Mr. Lachance] +22,15,32,[Cpl. Carl Hammaker] +8,0,7,[Emil] +4,116,151,[V] +127,121,167,[Taejo Togokahn] +13,11,24,[John Milton] +40,37,52,[Frank Cushman] +140,137,194,[Bob Zelnick] +42,130,185,[Jan Edgecomb] +3,9,10,[Morpheus] +93,92,120,[Dieter Dengler] +17,15,27,[Col. Nathan R. Jessup] +58,56,76,[The Tracker] +16,29,40,[Maverick] +74,73,93,[Suzy] \ No newline at end of file diff --git a/examples/src/main/resources/movies/directed.csv b/examples/src/main/resources/movies/directed.csv new file mode 100644 index 0000000000000..0a2fd43f93e20 --- /dev/null +++ b/examples/src/main/resources/movies/directed.csv @@ -0,0 +1,45 @@ +source,target,id +72,73,97 +25,116,156 +71,85,113 +136,130,190 +158,157,227 +5,0,4 +96,159,231 +5,105,142 +6,0,5 +55,52,72 +25,128,178 +6,105,143 +104,100,136 +5,10,19 +115,137,196 +6,10,20 +27,15,38 +61,56,78 +35,29,46 +27,81,106 +5,121,169 +66,62,83 +6,121,170 +153,152,218 +59,92,123 +151,150,215 +91,87,118 +45,37,57 +96,95,127 +156,154,222 +99,97,131 +79,78,101 +5,9,12 +142,141,201 +6,9,13 +27,46,67 +108,105,141 +115,144,207 +149,147,212 +166,162,240 +115,111,150 +14,11,25 +151,161,233 +72,67,90 \ No newline at end of file diff --git a/examples/src/main/resources/movies/follows.csv b/examples/src/main/resources/movies/follows.csv new file mode 100644 index 0000000000000..7d3f2111949ed --- /dev/null +++ b/examples/src/main/resources/movies/follows.csv @@ -0,0 +1,4 @@ +source,target,id +167,168,243 +170,169,241 +168,169,242 \ No newline at end of file diff --git a/examples/src/main/resources/movies/movies.csv b/examples/src/main/resources/movies/movies.csv new file mode 100644 index 0000000000000..de2be10ef9039 --- /dev/null +++ b/examples/src/main/resources/movies/movies.csv @@ -0,0 +1,39 @@ +id,released,tagline,title +0,1999,Welcome to the Real World,The Matrix +9,2003,Free your mind,The Matrix Reloaded +10,2003,Everything that has a beginning has an end,The Matrix Revolutions +11,1997,Evil has its winning ways,The Devil's Advocate +15,1992,"In the heart of the nation's capital, in a courthouse of the U.S. government, one man will stop at nothing to keep his honor, and one will stop at nothing to find the truth.",A Few Good Men +29,1986,"I feel the need, the need for speed.",Top Gun +37,2000,The rest of his life begins now.,Jerry Maguire +46,1986,"For some, it's the last real taste of innocence, and the first real taste of life. But for everyone, it's the time that memories are made of.",Stand By Me +52,1997,A comedy from the heart that goes for the throat.,As Good as It Gets +56,1998,After life there is more. The end is just the beginning.,What Dreams May Come +62,1999,First loves last. Forever.,Snow Falling on Cedars +67,1998,At odds in life... in love on-line.,You've Got Mail +73,1993,"What if someone you never met, someone you never saw, someone you never knew was the only someone for you?",Sleepless in Seattle +78,1990,"A story of love, lava and burning desire.",Joe Versus the Volcano +81,1998,Can two friends sleep together and still love each other in the morning?,When Harry Met Sally +85,1996,In every life there comes a time when that thing you dream becomes that thing you do,That Thing You Do +87,2000,"Pain heals, Chicks dig scars... Glory lasts forever",The Replacements +92,2006,Based on the extraordinary true story of one man's fight for freedom,RescueDawn +95,1996,Come as you are,The Birdcage +97,1992,"It's a hell of a thing, killing a man",Unforgiven +100,1995,The hottest data on earth. In the coolest head in town,Johnny Mnemonic +105,2012,Everything is connected,Cloud Atlas +111,2006,Break The Codes,The Da Vinci Code +116,2006,Freedom! Forever!,V for Vendetta +121,2008,Speed has no limits,Speed Racer +128,2009,Prepare to enter a secret world of assassins,Ninja Assassin +130,1999,Walk a mile you'll never forget.,The Green Mile +137,2008,400 million people were waiting for the truth.,Frost/Nixon +141,1992,He didn't want law. He wanted justice.,Hoffa +144,1995,"Houston, we have a problem.",Apollo 13 +147,1996,Don't Breathe. Don't Look Back.,Twister +150,2000,"At the edge of the world, his journey begins.",Cast Away +152,1975,"If he's crazy, what does that make you?",One Flew Over the Cuckoo's Nest +154,2003,null,Something's Gotta Give +157,1999,One robot's 200 year journey to become an ordinary man.,Bicentennial Man +159,2007,A stiff drink. A little mascara. A lot of nerve. Who said they couldn't bring down the Soviet empire.,Charlie Wilson's War +161,2004,This Holiday Season… Believe,The Polar Express +162,1992,Once in a lifetime you get a chance to do something different.,A League of Their Own \ No newline at end of file diff --git a/examples/src/main/resources/movies/persons.csv b/examples/src/main/resources/movies/persons.csv new file mode 100644 index 0000000000000..539f43ee0dc6d --- /dev/null +++ b/examples/src/main/resources/movies/persons.csv @@ -0,0 +1,134 @@ +id,name,born +1,Keanu Reeves,1964 +2,Carrie-Anne Moss,1967 +3,Laurence Fishburne,1961 +4,Hugo Weaving,1960 +5,Lilly Wachowski,1967 +6,Lana Wachowski,1965 +7,Joel Silver,1952 +8,Emil Eifrem,1978 +12,Charlize Theron,1975 +13,Al Pacino,1940 +14,Taylor Hackford,1944 +16,Tom Cruise,1962 +17,Jack Nicholson,1937 +18,Demi Moore,1962 +19,Kevin Bacon,1958 +20,Kiefer Sutherland,1966 +21,Noah Wyle,1971 +22,Cuba Gooding Jr.,1968 +23,Kevin Pollak,1957 +24,J.T. Walsh,1943 +25,James Marshall,1967 +26,Christopher Guest,1948 +27,Rob Reiner,1947 +28,Aaron Sorkin,1961 +30,Kelly McGillis,1957 +31,Val Kilmer,1959 +32,Anthony Edwards,1962 +33,Tom Skerritt,1933 +34,Meg Ryan,1961 +35,Tony Scott,1944 +36,Jim Cash,1941 +38,Renee Zellweger,1969 +39,Kelly Preston,1962 +40,Jerry O'Connell,1974 +41,Jay Mohr,1970 +42,Bonnie Hunt,1961 +43,Regina King,1971 +44,Jonathan Lipnicki,1996 +45,Cameron Crowe,1957 +47,River Phoenix,1970 +48,Corey Feldman,1971 +49,Wil Wheaton,1972 +50,John Cusack,1966 +51,Marshall Bell,1942 +53,Helen Hunt,1963 +54,Greg Kinnear,1963 +55,James L. Brooks,1940 +57,Annabella Sciorra,1960 +58,Max von Sydow,1929 +59,Werner Herzog,1942 +60,Robin Williams,1951 +61,Vincent Ward,1956 +63,Ethan Hawke,1970 +64,Rick Yune,1971 +65,James Cromwell,1940 +66,Scott Hicks,1953 +68,Parker Posey,1968 +69,Dave Chappelle,1973 +70,Steve Zahn,1967 +71,Tom Hanks,1956 +72,Nora Ephron,1941 +74,Rita Wilson,1956 +75,Bill Pullman,1953 +76,Victor Garber,1949 +77,Rosie O'Donnell,1962 +79,John Patrick Stanley,1950 +80,Nathan Lane,1956 +82,Billy Crystal,1948 +83,Carrie Fisher,1956 +84,Bruno Kirby,1949 +86,Liv Tyler,1977 +88,Brooke Langton,1970 +89,Gene Hackman,1930 +90,Orlando Jones,1968 +91,Howard Deutch,1950 +93,Christian Bale,1974 +94,Zach Grenier,1954 +96,Mike Nichols,1931 +98,Richard Harris,1930 +99,Clint Eastwood,1930 +101,Takeshi Kitano,1947 +102,Dina Meyer,1968 +103,Ice-T,1958 +104,Robert Longo,1953 +106,Halle Berry,1966 +107,Jim Broadbent,1949 +108,Tom Tykwer,1965 +109,David Mitchell,1969 +110,Stefan Arndt,1961 +112,Ian McKellen,1939 +113,Audrey Tautou,1976 +114,Paul Bettany,1971 +115,Ron Howard,1954 +117,Natalie Portman,1981 +118,Stephen Rea,1946 +119,John Hurt,1940 +120,Ben Miles,1967 +122,Emile Hirsch,1985 +123,John Goodman,1960 +124,Susan Sarandon,1946 +125,Matthew Fox,1966 +126,Christina Ricci,1980 +127,Rain,1982 +129,Naomie Harris,null +131,Michael Clarke Duncan,1957 +132,David Morse,1953 +133,Sam Rockwell,1968 +134,Gary Sinise,1955 +135,Patricia Clarkson,1959 +136,Frank Darabont,1959 +138,Frank Langella,1938 +139,Michael Sheen,1969 +140,Oliver Platt,1960 +142,Danny DeVito,1944 +143,John C. Reilly,1965 +145,Ed Harris,1950 +146,Bill Paxton,1955 +148,Philip Seymour Hoffman,1967 +149,Jan de Bont,1943 +151,Robert Zemeckis,1951 +153,Milos Forman,1932 +155,Diane Keaton,1946 +156,Nancy Meyers,1949 +158,Chris Columbus,1958 +160,Julia Roberts,1967 +163,Madonna,1954 +164,Geena Davis,1956 +165,Lori Petty,1963 +166,Penny Marshall,1943 +167,Paul Blythe,null +168,Angela Scope,null +169,Jessica Thompson,null +170,James Thompson,null \ No newline at end of file diff --git a/examples/src/main/resources/movies/produced.csv b/examples/src/main/resources/movies/produced.csv new file mode 100644 index 0000000000000..0de5ae69c3af0 --- /dev/null +++ b/examples/src/main/resources/movies/produced.csv @@ -0,0 +1,16 @@ +source,target,id +45,37,58 +72,81,108 +156,154,223 +7,9,14 +7,0,6 +5,116,157 +6,116,158 +7,10,21 +7,116,159 +27,81,107 +110,105,145 +5,128,179 +6,128,180 +7,121,173 +7,128,181 \ No newline at end of file diff --git a/examples/src/main/resources/movies/reviewed.csv b/examples/src/main/resources/movies/reviewed.csv new file mode 100644 index 0000000000000..3a5b57938f38a --- /dev/null +++ b/examples/src/main/resources/movies/reviewed.csv @@ -0,0 +1,10 @@ +source,target,id,rating,summary +169,37,252,92,You had me at Jerry +170,87,246,100,The coolest football movie ever +168,87,247,62,Pretty funny at times +169,87,245,65,"Silly, but fun" +169,95,249,45,Slapstick redeemed only by the Robin Williams and Gene Hackman's stellar performances +169,97,248,85,"Dark, but compelling" +169,105,244,95,An amazing journey +169,111,250,68,A solid romp +170,111,251,65,"Fun, but a little far fetched" \ No newline at end of file diff --git a/examples/src/main/resources/movies/wrote.csv b/examples/src/main/resources/movies/wrote.csv new file mode 100644 index 0000000000000..c5a1746e957ee --- /dev/null +++ b/examples/src/main/resources/movies/wrote.csv @@ -0,0 +1,11 @@ +source,target,id +156,154,224 +5,116,160 +45,37,59 +6,116,161 +28,15,39 +72,81,109 +109,105,144 +36,29,47 +5,121,171 +6,121,172 \ No newline at end of file diff --git a/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala new file mode 100644 index 0000000000000..fbaf156bcaaa2 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -0,0 +1,160 @@ +/* + * 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.spark.examples.graph + +// $example on$ + +import org.apache.spark.cypher.SparkCypherSession +import org.apache.spark.cypher.adapters.SchemaAdapter +import org.apache.spark.graph.api.{NodeDataset, PropertyGraph, PropertyGraphSchema, RelationshipDataset} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.functions._ + +import scala.io.StdIn +// $example off$ + +object AdvancedQueriesExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder() + .appName(s"${this.getClass.getSimpleName}") + .config("spark.master", "local[*]") + .getOrCreate() + val sc = spark.sparkContext + sc.setLogLevel("ERROR") + + val csvConfig = Map("header" -> "true", "delimiter" -> ",", "inferSchema" -> "true") + val resourcePath = "examples/src/main/resources/movies" + + // Load node dfs + val moviesData = spark.read.options(csvConfig).csv(s"$resourcePath/movies.csv") + val personsData = spark.read.options(csvConfig).csv(s"$resourcePath/persons.csv") + + // Load edge dfs + val actedInDF = spark.read.options(csvConfig).csv(s"$resourcePath/acted_in.csv") + val stringToArrayColumn = split(regexp_replace(actedInDF.col("roles"), "\\[|\\]|\"", " "), ",") + val actedInData = actedInDF.withColumn("roles", stringToArrayColumn) + val directedData = spark.read.options(csvConfig).csv(s"$resourcePath/directed.csv") + val followsData = spark.read.options(csvConfig).csv(s"$resourcePath/follows.csv") + val producedData = spark.read.options(csvConfig).csv(s"$resourcePath/produced.csv") + val reviewedData = spark.read.options(csvConfig).csv(s"$resourcePath/reviewed.csv") + val wroteData = spark.read.options(csvConfig).csv(s"$resourcePath/wrote.csv") + + // Initialise a GraphSession + val cypherSession = SparkCypherSession.create(spark) + + // Create Node- and RelationshipFrames + val movieNodes: NodeDataset = NodeDataset.builder(moviesData) + .idColumn("id") + .labelSet(Array("Movie")) + .properties(Map("title" -> "title")) + .build() + val personNodes: NodeDataset = NodeDataset.builder(personsData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + + val actedInRelationships: RelationshipDataset = RelationshipDataset.builder(actedInData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("ACTED_IN") + .properties(Map("roles" -> "roles")) + .build() + val directedRelationships: RelationshipDataset = RelationshipDataset.builder(directedData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("DIRECTED") + .build() + val followsRelationships: RelationshipDataset = RelationshipDataset.builder(followsData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("FOLLOWS") + .build() + val producedRelationships: RelationshipDataset = RelationshipDataset.builder(producedData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("PRODUCED") + .build() + val reviewedRelationships: RelationshipDataset = RelationshipDataset.builder(reviewedData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("REVIEWED") + .properties(Map("rating" -> "rating")) + .build() + val wroteRelationships: RelationshipDataset = RelationshipDataset.builder(wroteData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("WROTE") + .build() + val relationships = Array( + actedInRelationships, directedRelationships, followsRelationships, + producedRelationships, reviewedRelationships, wroteRelationships + ) + + // Create a PropertyGraph + val graph: PropertyGraph = cypherSession.createGraph(Array(movieNodes, personNodes), relationships) + print(graph.schema) + StdIn.readLine("Press Enter to continue: ") + + val bestMovies = graph.cypher( + """ + |MATCH (:Person)-[r:REVIEWED]->(m:Movie) + |WITH DISTINCT id(m) AS id, m.title AS title, round(avg(r.rating)) AS avgRating + |RETURN title, avgRating + |ORDER BY avgRating DESC""".stripMargin) + + println("Best rated movies") + bestMovies.ds.show() + StdIn.readLine("Press Enter to continue: ") + + val multiActor = graph.cypher( + """ + |MATCH (p:Person)-[r:ACTED_IN]->(m:Movie) + |WHERE size(r.roles) > 1 + |RETURN p.name AS actor, m.title AS movie, r.roles AS roles + |ORDER BY size(roles) DESC + |""".stripMargin) + + println("Actors playing multiple roles in one movie") + multiActor.ds.show(false) + StdIn.readLine("Press Enter to continue: ") + + val nearKevinBacon = graph.cypher( + """ + |MATCH (bacon:Person {name: 'Kevin Bacon'})-[*1..3]-(hollywood) + |RETURN DISTINCT hollywood + """.stripMargin) + + println("""Movies and actors up to 3 hops away from Kevin Bacon""") + nearKevinBacon.ds.show() + + spark.stop() + } + + private def print(schema: PropertyGraphSchema): Unit = { + println(schema.asInstanceOf[SchemaAdapter].schema.pretty) + } +} + diff --git a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala new file mode 100644 index 0000000000000..da9889f13774e --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -0,0 +1,110 @@ +/* + * 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.spark.examples.graph + +// $example on$ + +import org.apache.spark.cypher.SparkCypherSession +import org.apache.spark.graph.api.{NodeDataset, PropertyGraph, RelationshipDataset} +import org.apache.spark.sql.{SaveMode, SparkSession} + +import scala.io.StdIn +// $example off$ + +object CompleteGraphAPIExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder() + .appName(s"${this.getClass.getSimpleName}") + .config("spark.master", "local[*]") + .getOrCreate() + val sc = spark.sparkContext + sc.setLogLevel("ERROR") + + val csvConfig = Map("header" -> "true", "delimiter" -> ",", "inferSchema" -> "true") + val resourcePath = "examples/src/main/resources/movies" + // Load node dfs and edge df + val moviesData = spark.read.options(csvConfig).csv(s"$resourcePath/movies.csv") + val personsData = spark.read.options(csvConfig).csv(s"$resourcePath/persons.csv") + val actedInData = spark.read.options(csvConfig).csv(s"$resourcePath/acted_in.csv") + + println("Movies") + moviesData.show(2, truncate = false) + println("Acted IN") + actedInData.show(2, truncate = false) + + StdIn.readLine("Press Enter to continue: ") + + // Initialise a GraphSession + val cypherSession = SparkCypherSession.create(spark) + + // Create Node- and RelationshipFrames + val moviesNodeFrame = NodeDataset.builder(moviesData) + .idColumn("id") + .labelSet(Array("Movie")) + .properties(Map("title" -> "title")) + .build() + val personsNodeFrame = NodeDataset.builder(personsData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + val actedInRelationshipFrame = RelationshipDataset.builder(actedInData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("ACTED_IN") + .build() + + // Create a PropertyGraph + val graph: PropertyGraph = cypherSession.createGraph( + Array(moviesNodeFrame, personsNodeFrame), + Array(actedInRelationshipFrame) + ) + + // Get existing node labels + val labelSet = graph.schema.labels + println(s"The graph contains nodes with the following labels: ${labelSet.mkString(",")}") + println() + StdIn.readLine("Press Enter to continue: ") + + val businessNodes = graph.nodeDataset(Array("Movie")) + businessNodes.ds.show() + StdIn.readLine("Press Enter to continue: ") + + // Run parameterised cypher query + val parameters = Map("name" -> "Tom Hanks") + val result = graph.cypher( + """ + |MATCH (p:Person {name: $name})-[:ACTED_IN]->(movie) + |RETURN p.name, movie.title + """.stripMargin, parameters) + println(s"Movies with ${parameters.get("name")}") + result.ds.show() + + // Store the PropertyGraph + val savePath = "examples/src/main/resources/exampleGraph/" + graph.write.mode(SaveMode.Overwrite).save(savePath) + + // Load the PropertyGraph + val importedGraph = cypherSession.read.load(savePath) + + spark.stop() + } +} + diff --git a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala new file mode 100644 index 0000000000000..631fe630e6e9d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala @@ -0,0 +1,63 @@ +/* + * 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.spark.examples.graph + +// $example on$ + +import org.apache.spark.cypher.SparkCypherSession +import org.apache.spark.graph.api.{CypherResult, CypherSession, PropertyGraph} +// $example off$ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.SparkSession + +object SimpleExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder() + .appName(s"${this.getClass.getSimpleName}") + .config("spark.master", "local[*]") + .getOrCreate() + + // Create node df and edge df + val nodeData: DataFrame = spark + .createDataFrame(Seq((0, "Alice", true), (1, "Bob", true))) + .toDF(CypherSession.ID_COLUMN, "name", ":Person") + val relationshipData: DataFrame = spark + .createDataFrame(Seq((0, 0, 1, true))) + .toDF(CypherSession.ID_COLUMN, CypherSession.SOURCE_ID_COLUMN, CypherSession.TARGET_ID_COLUMN, ":KNOWS") + + // Initialise a GraphSession + val cypherSession = SparkCypherSession.create(spark) + + // Create a PropertyGraph + val graph: PropertyGraph = cypherSession.createGraph(nodeData, relationshipData) + + // Run our first query + val result: CypherResult = graph.cypher( + """ + |MATCH (a:Person)-[r:KNOWS]->(:Person) + |RETURN a, r + """.stripMargin) + + // Print the result + result.ds.show() + + spark.stop() + } +} + diff --git a/graph/api/pom.xml b/graph/api/pom.xml index a18c5290195fe..1f9cbe9267816 100644 --- a/graph/api/pom.xml +++ b/graph/api/pom.xml @@ -41,11 +41,39 @@ spark-core_${scala.binary.version} ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + org.apache.spark spark-sql_${scala.binary.version} ${project.version} + test-jar + test + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/CypherResult.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherResult.scala new file mode 100644 index 0000000000000..a2fc4d231441e --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherResult.scala @@ -0,0 +1,46 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +/** + * Result of a Cypher query. + * + * Wraps a Dataset that contains the result rows. + * + * @since 3.0.0 + */ +@Evolving +trait CypherResult { + // Note that representing the CypherResult as a trait allows for future extensions + // (e.g. returning graphs in addition to tables). + + /** + * Contains the result rows. + * + * The column names are aligned with the return item names specified within the Cypher query, + * (e.g. `RETURN foo, bar AS baz` results in the columns `foo` and `baz`). + * + * @note Dot characters (i.e. `.`) within return item names are replaced by an underscore (`_`), + * (e.g. `MATCH (n:Person) RETURN n` results in the columns `n`, `n:Person` and `n_name`). + * @since 3.0.0 + */ + def ds: Dataset[Row] +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala new file mode 100644 index 0000000000000..c5bacb365107c --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala @@ -0,0 +1,293 @@ +/* + * 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.spark.graph.api + +import scala.collection.JavaConverters._ + +import org.slf4j.LoggerFactory + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.apache.spark.sql.types.{BooleanType, StructType} + +/** + * Contains constants used for convention based column naming. + */ +@Evolving +object CypherSession { + + /** + * Naming convention for identifier columns, both node and relationship identifiers. + */ + val ID_COLUMN = "$ID" + + /** + * Naming convention for relationship source identifier. + */ + val SOURCE_ID_COLUMN = "$SOURCE_ID" + + /** + * Naming convention for relationship target identifier. + */ + val TARGET_ID_COLUMN = "$TARGET_ID" + + /** + * Naming convention for node label prefixes. + */ + val LABEL_COLUMN_PREFIX = ":" + + /** + * Naming convention for relationship type prefixes. + */ + val REL_TYPE_COLUMN_PREFIX = ":" + + /** + * Extracts [[NodeDataset]]s from a [[Dataset]] using column name conventions. + * + * For information about naming conventions, see [[CypherSession.createGraph]]. + * + * @param nodes node dataset + * @since 3.0.0 + */ + def extractNodeDatasets(nodes: Dataset[Row]): Array[NodeDataset] = { + val labelColumns = nodes.columns.filter(_.startsWith(LABEL_COLUMN_PREFIX)).toSet + validateLabelOrRelTypeColumns(nodes.schema, labelColumns, LABEL_COLUMN_PREFIX) + + val nodeProperties = (nodes.columns.toSet - ID_COLUMN -- labelColumns) + .map(col => col -> col) + .toMap + + val labelCount = labelColumns.size + if (labelCount > 5) { + LoggerFactory.getLogger(CypherSession.getClass).warn( + s"$labelCount label columns will result in ${Math.pow(labelCount, 2)} node datasets.") + if (labelCount > 10) { + throw new IllegalArgumentException( + s"Expected number of label columns to be less than or equal to 10, was $labelCount.") + } + } + + val labelSets = labelColumns.subsets().toSet + + labelSets.map { labelSet => + val predicate = labelColumns + .map { labelColumn => + if (labelSet.contains(labelColumn)) { + nodes.col(labelColumn) + } else { + !nodes.col(labelColumn) + } + } + .reduce(_ && _) + + NodeDataset(nodes.filter(predicate), ID_COLUMN, labelSet.map(_.substring(1)), nodeProperties) + }.toArray + } + + /** + * Extracts [[RelationshipDataset]]s from a [[Dataset]] using column name conventions. + * + * For information about naming conventions, see [[CypherSession.createGraph]]. + * + * @param relationships relationship dataset + * @since 3.0.0 + */ + def extractRelationshipDatasets(relationships: Dataset[Row]): Array[RelationshipDataset] = { + val relColumns = relationships.columns.toSet + val relTypeColumns = relColumns.filter(_.startsWith(REL_TYPE_COLUMN_PREFIX)) + validateLabelOrRelTypeColumns(relationships.schema, relTypeColumns, REL_TYPE_COLUMN_PREFIX) + val idColumns = Set(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN) + val propertyColumns = relColumns -- idColumns -- relTypeColumns + val relProperties = propertyColumns.map(col => col -> col).toMap + relTypeColumns.map { relTypeColumn => + val predicate = relationships.col(relTypeColumn) + // TODO: Make sure that each row represents a single relationship type + // see https://issues.apache.org/jira/browse/SPARK-29480 + RelationshipDataset( + relationships.filter(predicate), + ID_COLUMN, + SOURCE_ID_COLUMN, + TARGET_ID_COLUMN, + relTypeColumn.substring(1), + relProperties) + }.toArray + } + + /** + * Validates if the given columns fulfil specific constraints for + * representing node labels or relationship types. + * + * In particular, we check if the columns store boolean values and that + * the column name represents a single node label or relationship type. + * + * @param schema Dataset schema + * @param columns columns to validate + * @param prefix node label or relationship type prefix + */ + private def validateLabelOrRelTypeColumns( + schema: StructType, + columns: Set[String], + prefix: String): Unit = { + schema.fields.filter(f => columns.contains(f.name)).foreach(field => { + if (field.dataType != BooleanType) { + throw new IllegalArgumentException(s"Column ${field.name} must be of type BooleanType.") + } + }) + columns.foreach(typeColumn => { + if (typeColumn.sliding(prefix.length).count(_ == prefix) != 1) { + throw new IllegalArgumentException( + s"Type column $typeColumn must contain exactly one type." + ) + } + }) + } + +} + +/** + * A CypherSession allows for creating, storing and loading [[PropertyGraph]] instances as well as + * executing Cypher queries on them. + * + * Wraps a [[org.apache.spark.sql.SparkSession]]. + * + * @since 3.0.0 + */ +@Evolving +trait CypherSession { + + def sparkSession: SparkSession + + /** + * Executes a Cypher query on the given input graph. + * + * @param graph [[PropertyGraph]] on which the query is executed + * @param query Cypher query to execute + * @since 3.0.0 + */ + def cypher(graph: PropertyGraph, query: String): CypherResult + + /** + * Executes a Cypher query on the given input graph. + * + * Note that queries can take optional parameters: + * + * {{{ + * Parameters: + * + * { + * "name" : "Alice" + * } + * + * Query: + * + * MATCH (n:Person) + * WHERE n.name = $name + * RETURN n + * }}} + * + * @param graph [[PropertyGraph]] on which the query is executed + * @param query Cypher query to execute + * @param parameters parameters used by the Cypher query + * @since 3.0.0 + */ + def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult + + /** + * Executes a Cypher query on the given input graph. + * + * Note that queries can take optional parameters: + * + * {{{ + * Parameters: + * + * { + * "name" : "Alice" + * } + * + * Query: + * + * MATCH (n:Person) + * WHERE n.name = $name + * RETURN n + * }}} + * + * @param graph [[PropertyGraph]] on which the query is executed + * @param query Cypher query to execute + * @param parameters parameters used by the Cypher query + * @since 3.0.0 + */ + def cypher( + graph: PropertyGraph, + query: String, + parameters: java.util.Map[String, Object]): CypherResult = { + cypher(graph, query, parameters.asScala.toMap) + } + + /** + * Creates a [[PropertyGraph]] from a sequence of [[NodeDataset]]s and [[RelationshipDataset]]s. + * At least one [[NodeDataset]] has to be provided. + * + * For each label set and relationship type there can be at most one [[NodeDataset]] and at most + * one [[RelationshipDataset]], respectively. + * + * @param nodes NodeDataset that define the nodes in the graph + * @param relationships RelationshipDataset that define the relationships in the graph + * @since 3.0.0 + */ + def createGraph( + nodes: Array[NodeDataset], + relationships: Array[RelationshipDataset]): PropertyGraph + + /** + * Creates a [[PropertyGraph]] from nodes and relationships. + * + * The given dataset needs to adhere to the following column naming conventions: + * + * {{{ + * Id column: `$ID` (nodes and relationships) + * SourceId column: `$SOURCE_ID` (relationships) + * TargetId column: `$TARGET_ID` (relationships) + * + * Label columns: `:{LABEL_NAME}` (nodes) + * RelType columns: `:{REL_TYPE}` (relationships) + * + * Property columns: `{Property_Key}` (nodes and relationships) + * }}} + * + * @note It is recommended to cache the input datasets if they represent multiple label sets and + * relationship types. + * + * @see [[CypherSession]] + * @param nodes node dataset + * @param relationships relationship dataset + * @since 3.0.0 + */ + def createGraph(nodes: Dataset[Row], relationships: Dataset[Row]): PropertyGraph = { + val nodeFrames = CypherSession.extractNodeDatasets(nodes) + val relationshipFrames = CypherSession.extractRelationshipDatasets(relationships) + createGraph(nodeFrames, relationshipFrames) + } + + /** + * Returns a [[PropertyGraphReader]] that can be used to read data in as a `PropertyGraph`. + * + * @since 3.0.0 + */ + def read: PropertyGraphReader + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataset.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataset.scala new file mode 100644 index 0000000000000..dd7d915f75fb6 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataset.scala @@ -0,0 +1,61 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +/** + * A [[PropertyGraph]] component containing graph elements; either nodes or relationships. + * A GraphElementDataset wraps a Dataset and describes how it maps to graph elements. + * + * @since 3.0.0 + */ +@Evolving +abstract class GraphElementDataset { + + /** + * Initial Dataset that can still contain unmapped, arbitrarily ordered columns. + * + * @since 3.0.0 + */ + def ds: Dataset[Row] + + /** + * Name of the column that contains the graph element identifier. + * + * @since 3.0.0 + */ + def idColumn: String + + /** + * Name of all columns that contain graph element identifiers. + * + * @since 3.0.0 + */ + def idColumns: Array[String] = Array(idColumn) + + /** + * Mapping from graph element property keys to the columns that contain the corresponding property + * values. + * + * @since 3.0.0 + */ + def propertyColumns: Map[String, String] + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDataset.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDataset.scala new file mode 100644 index 0000000000000..51a07d0eb6b34 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDataset.scala @@ -0,0 +1,54 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +object NodeDataset { + + /** + * Returns a [[NodeDatasetBuilder]] that can be used to construct a [[NodeDataset]]. + * + * @param ds Dataset containing a single node in each row + * @since 3.0.0 + */ + def builder(ds: Dataset[Row]): NodeDatasetBuilder = + new NodeDatasetBuilder(ds) + +} + +/** + * Describes how to map a Dataset to nodes. + * + * Each row in the Dataset represents a node which has exactly the labels defined by the given + * label set. + * + * @param ds Dataset containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labelSet labels that are assigned to all nodes + * @param propertyColumns mapping from property keys to corresponding columns + * @since 3.0.0 + */ +@Evolving +case class NodeDataset private[graph]( + ds: Dataset[Row], + idColumn: String, + labelSet: Set[String], + propertyColumns: Map[String, String]) + extends GraphElementDataset diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDatasetBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDatasetBuilder.scala new file mode 100644 index 0000000000000..543fe5ffb4974 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDatasetBuilder.scala @@ -0,0 +1,86 @@ +/* + * 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.spark.graph.api + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +/** + * A builder for [[NodeDataset]]. + * + * @param ds Dataset containing a single node in each row + * @since 3.0.0 + */ +@Evolving +final class NodeDatasetBuilder(val ds: Dataset[Row]) { + + private var idColumn: String = CypherSession.ID_COLUMN + private var labelSet: Set[String] = Set.empty + private var properties: Map[String, String] = Map.empty + + /** + * @param idColumn column that contains the node identifier + * @since 3.0.0 + */ + def idColumn(idColumn: String): NodeDatasetBuilder = { + if (idColumn.isEmpty) { + throw new IllegalArgumentException("idColumn must not be empty") + } + this.idColumn = idColumn; + this + } + + /** + * @param labelSet labels that are assigned to all nodes + * @since 3.0.0 + */ + def labelSet(labelSet: Array[String]): NodeDatasetBuilder = { + this.labelSet = labelSet.toSet + this + } + + /** + * @param properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def properties(properties: Map[String, String]): NodeDatasetBuilder = { + this.properties = properties + this + } + + /** + * @param properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def properties(properties: java.util.Map[String, String]): NodeDatasetBuilder = { + this.properties = properties.asScala.toMap + this + } + + /** + * Creates a `NodeDataset` from the specified builder parameters. + * + * @since 3.0.0 + */ + def build(): NodeDataset = { + NodeDataset(ds, idColumn, labelSet, properties) + } + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraph.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraph.scala new file mode 100644 index 0000000000000..de58187472129 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraph.scala @@ -0,0 +1,140 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +/** + * A Property Graph as defined by the openCypher Property Graph Data Model. + * + * A graph is always tied to and managed by a [[CypherSession]]. + * The lifetime of a graph is bound by the session lifetime. + * + * @see openCypher project + * @see Property Graph Model + * @since 3.0.0 + */ +@Evolving +abstract class PropertyGraph { + + /** + * The schema (graph type) describes the structure of this graph. + * + * @since 3.0.0 + */ + def schema: PropertyGraphSchema + + /** + * The session in which this graph is managed. + * + * @since 3.0.0 + */ + def cypherSession: CypherSession + + /** + * Executes a Cypher query in the session that manages this graph, using this graph as + * the input graph. + * + * @param query Cypher query to execute + * @since 3.0.0 + */ + def cypher(query: String): CypherResult = cypher(query, Map.empty[String, Any]) + + /** + * Executes a Cypher query in the session that manages this graph, using this graph as + * the input graph. + * + * @param query Cypher query to execute + * @param parameters parameters used by the Cypher query + * @since 3.0.0 + */ + def cypher(query: String, parameters: Map[String, Any]): CypherResult = + cypherSession.cypher(this, query, parameters) + + /** + * Executes a Cypher query in the [[CypherSession]] that manages this graph, using this graph as + * the input graph. + * + * @param query Cypher query to execute + * @param parameters parameters used by the Cypher query + * @since 3.0.0 + */ + def cypher(query: String, parameters: java.util.Map[String, Object]): CypherResult = + cypherSession.cypher(this, query, parameters) + + /** + * Returns the [[NodeDataset]] for a given node label set. + * + * @param labelSet Label set used for NodeDataset lookup + * @return NodeDataset for the given label set + * @since 3.0.0 + */ + def nodeDataset(labelSet: Array[String]): NodeDataset + + /** + * Returns the [[RelationshipDataset]] for a given relationship type. + * + * @param relationshipType Relationship type used for RelationshipDataset lookup + * @return RelationshipDataset for the given relationship type + * @since 3.0.0 + */ + def relationshipDataset(relationshipType: String): RelationshipDataset + + /** + * Returns a Dataset that contains a row for each node in this graph. + * + * The Dataset adheres to the following column naming conventions: + * + * {{{ + * Id column: `$ID` + * Label columns: `:{LABEL_NAME}` + * Property columns: `{Property_Key}` + * }}} + * + * @see `org.apache.spark.graph.api.CypherSession.createGraph(nodes, relationships)` + * @since 3.0.0 + */ + def nodes: Dataset[Row] + + /** + * Returns a Dataset that contains a row for each relationship in this + * graph. + * + * The Dataset adheres to column naming conventions: + * + * {{{ + * Id column: `$ID` + * SourceId column: `$SOURCE_ID` + * TargetId column: `$TARGET_ID` + * RelType columns: `:{REL_TYPE}` + * Property columns: `{Property_Key}` + * }}} + * + * @see `org.apache.spark.graph.api.CypherSession.createGraph(nodes, relationships)` + * @since 3.0.0 + */ + def relationships: Dataset[Row] + + /** + * Interface for saving this `PropertyGraph` out into external storage. + * + * @since 3.0.0 + */ + def write: PropertyGraphWriter +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphReader.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphReader.scala new file mode 100644 index 0000000000000..258a8238d45d7 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphReader.scala @@ -0,0 +1,45 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving + +@Evolving +abstract class PropertyGraphReader(session: CypherSession) { + + protected var format: String = + session.sparkSession.sessionState.conf.defaultDataSourceName + + /** + * Specifies the underlying input data format. Built-in options include "parquet", "json", etc. + * + * @since 3.0.0 + */ + def format(format: String): PropertyGraphReader = { + this.format = format + this + } + + /** + * Loads input in as a `PropertyGraph` from the specified path. + * + * @since 3.0.0 + */ + def load(path: String): PropertyGraph + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphSchema.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphSchema.scala new file mode 100644 index 0000000000000..52c690fb71a6e --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphSchema.scala @@ -0,0 +1,50 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving + +/** + * Describes the structure of a [[PropertyGraph]]. + * + * @since 3.0.0 + */ +@Evolving +trait PropertyGraphSchema { + + /** + * Returns all labels occurring on any node in the graph. + * + * @since 3.0.0 + */ + def labels: Array[String] = labelSets.flatten + + /** + * Returns all distinct label sets occurring on nodes in the graph. + * + * @since 3.0.0 + */ + def labelSets: Array[Array[String]] + + /** + * Returns all relationship types occurring on relationships in the graph. + * + * @since 3.0.0 + */ + def relationshipTypes: Array[String] +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphWriter.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphWriter.scala new file mode 100644 index 0000000000000..9fba6bcc1067a --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphWriter.scala @@ -0,0 +1,92 @@ +/* + * 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.spark.graph.api + +import java.util.Locale + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.SaveMode + +@Evolving +abstract class PropertyGraphWriter(val graph: PropertyGraph) { + + protected var saveMode: SaveMode = SaveMode.ErrorIfExists + protected var format: String = + graph.cypherSession.sparkSession.sessionState.conf.defaultDataSourceName + + /** + * Specifies the behavior when the graph already exists. Options include: + * + *

+ * When writing the default option is `ErrorIfExists`. + * + * @since 3.0.0 + */ + def mode(mode: SaveMode): PropertyGraphWriter = { + mode match { + case SaveMode.Append => + throw new IllegalArgumentException(s"Unsupported save mode: $mode. " + + "Accepted save modes are 'overwrite', 'ignore', 'error', 'errorifexists'.") + case _ => + this.saveMode = mode + } + this + } + + /** + * Specifies the behavior when the graph already exists. Options include: + *

+ * + * @since 3.0.0 + */ + def mode(saveMode: String): PropertyGraphWriter = { + saveMode.toLowerCase(Locale.ROOT) match { + case "overwrite" => mode(SaveMode.Overwrite) + case "ignore" => mode(SaveMode.Ignore) + case "error" | "errorifexists" | "default" => mode(SaveMode.ErrorIfExists) + case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " + + "Accepted save modes are 'overwrite', 'ignore', 'error', 'errorifexists'.") + } + } + + /** + * Specifies the underlying output data format. Built-in options include "parquet", "json", etc. + * + * @since 3.0.0 + */ + def format(format: String): PropertyGraphWriter = { + this.format = format + this + } + + /** + * Saves the content of the `PropertyGraph` at the specified path. + * + * @since 3.0.0 + */ + def save(path: String): Unit + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDataset.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDataset.scala new file mode 100644 index 0000000000000..ab4bb2b91e6f3 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDataset.scala @@ -0,0 +1,62 @@ +/* + * 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.spark.graph.api + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +object RelationshipDataset { + + /** + * Returns a [[RelationshipDatasetBuilder]] that can be used to construct + * a [[RelationshipDataset]]. + * + * @param ds Dataset containing a single relationship in each row + * @since 3.0.0 + */ + def builder(ds: Dataset[Row]): RelationshipDatasetBuilder = + new RelationshipDatasetBuilder(ds) + +} + +/** + * Describes how to map a Dataset to relationships. + * + * Each row in the Dataset represents a relationship with the given relationship type. + * + * @param ds Dataset containing a single relationship in each row + * @param idColumn column that contains the relationship identifier + * @param sourceIdColumn column that contains the source node identifier of the relationship + * @param targetIdColumn column that contains the target node identifier of the relationship + * @param relationshipType relationship type that is assigned to all relationships + * @param propertyColumns mapping from property keys to corresponding columns + * @since 3.0.0 + */ +@Evolving +case class RelationshipDataset private[graph]( + ds: Dataset[Row], + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + propertyColumns: Map[String, String]) + extends GraphElementDataset { + + override def idColumns: Array[String] = Array(idColumn, sourceIdColumn, targetIdColumn) + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDatasetBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDatasetBuilder.scala new file mode 100644 index 0000000000000..e0aee0f02fc77 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDatasetBuilder.scala @@ -0,0 +1,118 @@ +/* + * 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.spark.graph.api + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.{Dataset, Row} + +/** + * A builder for [[RelationshipDataset]]. + * + * @param ds Dataset containing a single relationship in each row + * @since 3.0.0 + */ +@Evolving +final class RelationshipDatasetBuilder(val ds: Dataset[Row]) { + + private var idColumn: String = CypherSession.ID_COLUMN + private var sourceIdColumn: String = CypherSession.SOURCE_ID_COLUMN + private var targetIdColumn: String = CypherSession.TARGET_ID_COLUMN + private var maybeRelationshipType: Option[String] = Option.empty + private var properties: Map[String, String] = Map.empty + + /** + * @param idColumn column that contains the relationship identifier + * @since 3.0.0 + */ + def idColumn(idColumn: String): RelationshipDatasetBuilder = { + if (idColumn.isEmpty) { + throw new IllegalArgumentException("idColumn must not be empty") + } + this.idColumn = idColumn; + this + } + + /** + * @param sourceIdColumn column that contains the source node identifier of the relationship + * @since 3.0.0 + */ + def sourceIdColumn(sourceIdColumn: String): RelationshipDatasetBuilder = { + if (sourceIdColumn.isEmpty) { + throw new IllegalArgumentException("sourceIdColumn must not be empty") + } + this.sourceIdColumn = sourceIdColumn; + this + } + + /** + * @param targetIdColumn column that contains the target node identifier of the relationship + * @since 3.0.0 + */ + def targetIdColumn(targetIdColumn: String): RelationshipDatasetBuilder = { + if (targetIdColumn.isEmpty) { + throw new IllegalArgumentException("targetIdColumn must not be empty") + } + this.targetIdColumn = targetIdColumn; + this + } + + /** + * @param relationshipType relationship type that is assigned to all relationships + * @since 3.0.0 + */ + def relationshipType(relationshipType: String): RelationshipDatasetBuilder = { + if (relationshipType.isEmpty) { + throw new IllegalArgumentException("Relationship type must not be empty") + } + this.maybeRelationshipType = Some(relationshipType) + this + } + + /** + * @param properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def properties(properties: Map[String, String]): RelationshipDatasetBuilder = { + this.properties = properties + this + } + + /** + * @param properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def properties(properties: java.util.Map[String, String]): RelationshipDatasetBuilder = { + this.properties = properties.asScala.toMap + this + } + + /** + * Creates a [[RelationshipDataset]] from the specified builder parameters. + * + * @since 3.0.0 + */ + def build(): RelationshipDataset = { + maybeRelationshipType match { + case Some(relType) => + RelationshipDataset(ds, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + case None => throw new IllegalArgumentException("Relationship type must be set.") + } + } +} diff --git a/graph/api/src/test/java/org/apache/spark/graph/api/JavaPropertyGraphSuite.java b/graph/api/src/test/java/org/apache/spark/graph/api/JavaPropertyGraphSuite.java new file mode 100644 index 0000000000000..a1f1b4b03140e --- /dev/null +++ b/graph/api/src/test/java/org/apache/spark/graph/api/JavaPropertyGraphSuite.java @@ -0,0 +1,113 @@ +/* + * 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.spark.graph.api; + +import com.google.common.collect.Lists; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.spark.sql.types.DataTypes.*; + +public abstract class JavaPropertyGraphSuite implements Serializable { + private transient TestSparkSession spark; + private transient CypherSession cypherSession; + + /** + * This method is supposed to return an implementation of {@link CypherSession} + * from a module that depends on spark-graph-api (e.g. spark-cypher). + * + * This allows us to run the same tests on arbitrary implementations. + */ + abstract CypherSession getCypherSession(SparkSession sparkSession); + + @Before + public void setUp() { + spark = new TestSparkSession(); + cypherSession = getCypherSession(spark); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @Test + public void testCreateFromNodeDataset() { + StructType personSchema = createSchema( + Lists.newArrayList("id", "name"), + Lists.newArrayList(LongType, StringType)); + + List personData = Arrays.asList( + RowFactory.create(0L, "Alice"), + RowFactory.create(1L, "Bob")); + + StructType knowsSchema = createSchema( + Lists.newArrayList("id", "source", "target", "since"), + Lists.newArrayList(LongType, LongType, LongType, IntegerType)); + + List knowsData = Collections.singletonList(RowFactory.create(0L, 0L, 1L, 1984)); + + Dataset personDf = spark.createDataFrame(personData, personSchema); + NodeDataset personNodeDataset = NodeDataset.builder(personDf) + .idColumn("id") + .labelSet(new String[]{"Person"}) + .properties(Collections.singletonMap("name", "name")) + .build(); + + Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); + RelationshipDataset knowsRelDataset = RelationshipDataset.builder(knowsDf) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Collections.singletonMap("since", "since")) + .build(); + + + PropertyGraph graph = cypherSession.createGraph( + new NodeDataset[]{personNodeDataset}, + new RelationshipDataset[]{knowsRelDataset}); + List result = graph.nodes().collectAsList(); + Assert.assertEquals(2, result.size()); + } + + private StructType createSchema(List fieldNames, List dataTypes) { + List fields = new ArrayList<>(); + for (int i = 0; i < fieldNames.size(); i++) { + fields.add(createStructField(fieldNames.get(i), dataTypes.get(i), true)); + } + return createStructType(fields); + } +} diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 12b3832d9777a..d6cee744c2bab 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -47,22 +47,25 @@ ${project.version} - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} + org.apache.spark + spark-graph-api_${scala.binary.version} + ${project.version} + + + org.opencypher + okapi-shade + ${okapi.version} + + + org.apache.spark spark-graph-api_${scala.binary.version} ${project.version} + test-jar + test - - org.opencypher - okapi-shade - ${okapi.version} - - - org.apache.spark @@ -71,6 +74,7 @@ test-jar test + org.apache.spark spark-sql_${scala.binary.version} @@ -78,6 +82,7 @@ test-jar test + org.apache.spark spark-catalyst_${scala.binary.version} @@ -85,12 +90,14 @@ test-jar test + org.opencypher okapi-tck ${okapi.version} test + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala new file mode 100644 index 0000000000000..02def3ee8158c --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala @@ -0,0 +1,72 @@ +/* + * 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.spark.cypher + +import org.apache.spark.cypher.SparkCypherEntity._ +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, Node, Relationship} + +object SparkCypherEntity { + + implicit class RichId(id: Seq[Byte]) { + + def toHex: String = s"0x${id.map(id => "%02X".format(id)).mkString}" + + } +} + +case class SparkCypherNode( + override val id: Seq[Byte], + override val labels: Set[String] = Set.empty, + override val properties: CypherMap = CypherMap.empty +) extends Node[Seq[Byte]] { + + override type I = SparkCypherNode + + override def copy( + id: Seq[Byte] = id, + labels: Set[String] = labels, + properties: CypherMap = properties + ): SparkCypherNode = { + SparkCypherNode(id, labels, properties) + } + + override def toString: String = s"${getClass.getSimpleName}(id=${id.toHex}, labels=$labels, properties=$properties)" +} + +case class SparkCypherRelationship( + override val id: Seq[Byte], + override val startId: Seq[Byte], + override val endId: Seq[Byte], + override val relType: String, + override val properties: CypherMap = CypherMap.empty +) extends Relationship[Seq[Byte]] { + + override type I = SparkCypherRelationship + + override def copy( + id: Seq[Byte] = id, + startId: Seq[Byte] = startId, + endId: Seq[Byte] = endId, + relType: String = relType, + properties: CypherMap = properties + ): SparkCypherRelationship = SparkCypherRelationship(id, startId, endId, relType, properties) + + override def toString: String = s"${getClass.getSimpleName}(id=${id.toHex}, startId=${startId.toHex}, endId=${endId.toHex}, relType=$relType, properties=$properties)" + +} \ No newline at end of file diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala new file mode 100644 index 0000000000000..cb7bc0aba3ed2 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala @@ -0,0 +1,152 @@ +/* + * 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.spark.cypher + +import org.apache.spark.cypher.conversions.TypeConversions._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} +import org.apache.spark.sql.{Column, Dataset, Row, functions} +import org.opencypher.okapi.api.value.CypherValue.{CypherList, CypherMap, CypherValue} +import org.opencypher.okapi.impl.exception.IllegalArgumentException +import org.opencypher.okapi.ir.api.expr.Expr +import org.opencypher.okapi.relational.impl.table.RecordHeader + +import scala.reflect.runtime.universe.TypeTag + +object SparkCypherFunctions { + + val NULL_LIT: Column = lit(null) + val TRUE_LIT: Column = lit(true) + val FALSE_LIT: Column = lit(false) + val ONE_LIT: Column = lit(1) + val E_LIT: Column = lit(Math.E) + val PI_LIT: Column = lit(Math.PI) + // See: https://issues.apache.org/jira/browse/SPARK-20193 + val EMPTY_STRUCT: Column = udf(() => new GenericRowWithSchema(Array(), StructType(Nil)), StructType(Nil))() + + implicit class RichColumn(column: Column) { + + /** + * This is a copy of {{{org.apache.spark.sql.Column#getItem}}}. The original method only allows fixed + * values (Int, or String) as index although the underlying implementation seem capable of processing arbitrary + * expressions. This method exposes these features + */ + def get(idx: Column): Column = + new Column(UnresolvedExtractValue(column.expr, idx.expr)) + } + + def list_slice(list: Column, maybeFrom: Option[Column], maybeTo: Option[Column]): Column = { + val start = maybeFrom.map(_ + ONE_LIT).getOrElse(ONE_LIT) + val length = (maybeTo.getOrElse(size(list)) - start) + ONE_LIT + new Column(Slice(list.expr, start.expr, length.expr)) + } + + /** + * Alternative version of `array_contains` that takes a column as the value. + */ + def array_contains(column: Column, value: Column): Column = + new Column(ArrayContains(column.expr, value.expr)) + + def hash64(columns: Column*): Column = + new Column(new XxHash64(columns.map(_.expr))) + + def regex_match(text: Column, pattern: Column): Column = new Column(RLike(text.expr, pattern.expr)) + + def get_array_item(array: Column, index: Int): Column = { + new Column(GetArrayItem(array.expr, functions.lit(index).expr)) + } + + private val x: NamedLambdaVariable = NamedLambdaVariable("x", StructType(Seq(StructField("item", StringType), StructField("flag", BooleanType))), nullable = false) + private val TRUE_EXPR: Expression = functions.lit(true).expr + + def filter_true[T: TypeTag](items: Seq[T], mask: Seq[Column]): Column = { + filter_with_mask(items, mask, LambdaFunction(EqualTo(GetStructField(x, 1), TRUE_EXPR), Seq(x), hidden = false)) + } + + def filter_not_null[T: TypeTag](items: Seq[T], mask: Seq[Column]): Column = { + filter_with_mask(items, mask, LambdaFunction(IsNotNull(GetStructField(x, 1)), Seq(x), hidden = false)) + } + + def make_big_decimal(unscaledVal: Column, precision: Int, scale: Int): Column = { + new Column(MakeDecimal(unscaledVal.expr, precision, scale)) + } + + private def filter_with_mask[T: TypeTag](items: Seq[T], mask: Seq[Column], predicate: LambdaFunction): Column = { + require(items.size == mask.size, s"Array filtering requires for the items and the mask to have the same length.") + if (items.isEmpty) { + functions.array() + } else { + val itemLiterals = functions.array(items.map(functions.typedLit): _*) + val zippedArray = functions.arrays_zip(itemLiterals, functions.array(mask: _*)) + val filtered = ArrayFilter(zippedArray.expr, predicate) + val transform = ArrayTransform(filtered, LambdaFunction(GetStructField(x, 0), Seq(x), hidden = false)) + new Column(transform) + } + } + + // See: https://issues.apache.org/jira/browse/SPARK-20193 + def create_struct(structColumns: Seq[Column]): Column = { + if (structColumns.isEmpty) EMPTY_STRUCT + else struct(structColumns: _*) + } + + def switch(branches: Seq[(Column, Column)], maybeDefault: Option[Column]): Column = { + new Column(CaseWhen(branches.map { case (c, v) => c.expr -> v.expr } , maybeDefault.map(_.expr))) + } + + /** + * Alternative version of {{{org.apache.spark.sql.functions.translate}}} that takes {{{org.apache.spark.sql.Column}}}s for search and replace strings. + */ + def translate(src: Column, matchingString: Column, replaceString: Column): Column = { + new Column(StringTranslate(src.expr, matchingString.expr, replaceString.expr)) + } + + def column_for(expr: Expr)(implicit header: RecordHeader, ds: Dataset[Row]): Column = { + val columnName = header.getColumn(expr).getOrElse(throw IllegalArgumentException( + expected = s"Expression in ${header.expressions.mkString("[", ", ", "]")}", + actual = expr) + ) + if (ds.columns.contains(columnName)) { + ds.col(columnName) + } else { + NULL_LIT + } + } + + implicit class CypherValueConversion(val v: CypherValue) extends AnyVal { + + def toSparkLiteral: Column = { + v.cypherType.ensureSparkCompatible() + v match { + case list: CypherList => array(list.value.map(_.toSparkLiteral): _*) + case map: CypherMap => create_struct( + map.value.map { case (key, value) => + value.toSparkLiteral.as(key.toString) + }.toSeq + ) + case _ => lit(v.unwrap) + } + } + + } + +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala new file mode 100644 index 0000000000000..981931e593c1b --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala @@ -0,0 +1,120 @@ +/* + * 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.spark.cypher + +import java.util.Collections + +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.conversions.CypherValueEncoders._ +import org.apache.spark.cypher.conversions.RowConversion +import org.apache.spark.cypher.conversions.TypeConversions._ +import org.apache.spark.sql._ +import org.opencypher.okapi.api.types._ +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherValue} +import org.opencypher.okapi.relational.api.io.ElementTable +import org.opencypher.okapi.relational.api.table.{RelationalCypherRecords, RelationalCypherRecordsFactory} +import org.opencypher.okapi.relational.impl.table._ + +import scala.collection.JavaConverters._ + +case class SparkCypherRecordsFactory()(implicit caps: SparkCypherSession) extends RelationalCypherRecordsFactory[DataFrameTable] { + + override type Records = SparkCypherRecords + + override def unit(): SparkCypherRecords = { + val initialDataFrame = caps.sparkSession.createDataFrame(Seq(EmptyRow())) + SparkCypherRecords(RecordHeader.empty, initialDataFrame) + } + + override def empty(initialHeader: RecordHeader = RecordHeader.empty): SparkCypherRecords = { + val initialSparkStructType = initialHeader.toStructType + val initialDataFrame = caps.sparkSession.createDataFrame(Collections.emptyList[Row](), initialSparkStructType) + SparkCypherRecords(initialHeader, initialDataFrame) + } + + override def fromElementTable(entityTable: ElementTable[DataFrameTable]): SparkCypherRecords = { + SparkCypherRecords(entityTable.header, entityTable.table.df) + } + + override def from( + header: RecordHeader, + table: DataFrameTable, + maybeDisplayNames: Option[Seq[String]] + ): SparkCypherRecords = { + val displayNames = maybeDisplayNames match { + case s@Some(_) => s + case None => Some(header.vars.map(_.withoutType).toSeq) + } + SparkCypherRecords(header, table, displayNames) + } + + private case class EmptyRow() +} + +case class SparkCypherRecords( + header: RecordHeader, + table: DataFrameTable, + override val logicalColumns: Option[Seq[String]] = None +)(implicit session: SparkCypherSession) extends RelationalCypherRecords[DataFrameTable] with RecordBehaviour { + override type Records = SparkCypherRecords + + def ds: Dataset[Row] = table.df + + override def cache(): SparkCypherRecords = { + ds.cache() + this + } + + override def toString: String = { + if (header.isEmpty) { + s"CAPSRecords.empty" + } else { + s"CAPSRecords(header: $header)" + } + } +} + +trait RecordBehaviour extends RelationalCypherRecords[DataFrameTable] { + + override lazy val columnType: Map[String, CypherType] = table.df.columnType + + override def rows: Iterator[String => CypherValue] = { + toLocalIterator.asScala.map(_.value) + } + + override def iterator: Iterator[CypherMap] = { + toLocalIterator.asScala + } + + def toLocalIterator: java.util.Iterator[CypherMap] = { + toCypherMaps.toLocalIterator() + } + + def foreachPartition(f: Iterator[CypherMap] => Unit): Unit = { + toCypherMaps.foreachPartition(f) + } + + override def collect: Array[CypherMap] = { + toCypherMaps.collect() + } + + def toCypherMaps: Dataset[CypherMap] = { + table.df.map(RowConversion(header.exprToColumn.toSeq)) + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala new file mode 100644 index 0000000000000..0f8e940be9f0b --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala @@ -0,0 +1,28 @@ +/* + * 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.spark.cypher + +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.graph.api.CypherResult +import org.apache.spark.sql.{Dataset, Row} +import org.opencypher.okapi.relational.api.table.RelationalCypherRecords + +case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable]) extends CypherResult { + override val ds: Dataset[Row] = relationalTable.table.df +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala new file mode 100644 index 0000000000000..0ee08823d50b5 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -0,0 +1,96 @@ +/* + * 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.spark.cypher + +import org.apache.spark.cypher.SparkCypherSession._ +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.adapters.RelationalGraphAdapter +import org.apache.spark.cypher.conversions.GraphElementFrameConversions.normalizeDf +import org.apache.spark.cypher.io.SparkCypherPropertyGraphReader +import org.apache.spark.graph.api._ +import org.apache.spark.sql.SparkSession +import org.opencypher.okapi.api.value.CypherValue.CypherMap +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, UnsupportedOperationException} +import org.opencypher.okapi.relational.api.graph.{RelationalCypherGraph, RelationalCypherGraphFactory, RelationalCypherSession} +import org.opencypher.okapi.relational.api.planning.RelationalCypherResult +import org.opencypher.okapi.relational.api.table.RelationalElementTableFactory + +object SparkCypherSession { + def create(implicit sparkSession: SparkSession): CypherSession = new SparkCypherSession(sparkSession) + + private[spark] def createInternal(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) + + private[cypher] def toRelationalGraph(graph: PropertyGraph): RelationalCypherGraph[DataFrameTable] = { + graph match { + case adapter: RelationalGraphAdapter => adapter.graph + case other => throw IllegalArgumentException( + expected = "A graph that has been created by `SparkCypherSession.createGraph`", + actual = other.getClass.getSimpleName + ) + } + } +} + +/** + * Default [[CypherSession]] implementation. + * + * This class is the main entry point for working with the spark-cypher module. + * It wraps a [[SparkSession]] and allows to run Cypher queries over graphs represented as [[org.apache.spark.sql.Dataset]]s. + */ +private[spark] class SparkCypherSession(override val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherSession { + + override type Result = RelationalCypherResult[DataFrameTable] + override type Records = SparkCypherRecords + + implicit def sparkCypherSession: SparkCypherSession = this + + override val records: SparkCypherRecordsFactory = SparkCypherRecordsFactory() + + override val graphs: RelationalCypherGraphFactory[DataFrameTable] = { + new RelationalCypherGraphFactory[DataFrameTable]() { + override implicit val session: RelationalCypherSession[DataFrameTable] = sparkCypherSession + } + } + + override def elementTables: RelationalElementTableFactory[DataFrameTable] = { + throw UnsupportedOperationException("Graph construction with `CONSTRUCT` is not supported in Cypher 9") + } + + override def createGraph(nodes: Array[NodeDataset], relationships: Array[RelationshipDataset]): PropertyGraph = { + require(nodes.groupBy(_.labelSet).forall(_._2.length == 1), + "There can be at most one NodeDataset per label set") + require(relationships.groupBy(_.relationshipType).forall(_._2.length == 1), + "There can be at most one RelationshipDataset per relationship type") + + val normalizedNodes = nodes.map(nf => nf.copy(ds = normalizeDf(nf))) + val normalizedRelationships = relationships.map(rf => rf.copy(ds = normalizeDf(rf))) + RelationalGraphAdapter(this, normalizedNodes, normalizedRelationships) + } + + def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty[String, Object]) + + override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { + val relationalGraph = toRelationalGraph(graph) + SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records) + } + + override def read(): SparkCypherPropertyGraphReader = + new SparkCypherPropertyGraphReader(this) + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala new file mode 100644 index 0000000000000..f98889ac2469d --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala @@ -0,0 +1,38 @@ +/* + * 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.spark.cypher + +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.opencypher.okapi.api.io.conversion.ElementMapping +import org.opencypher.okapi.relational.api.io.ElementTable + +case class SparkEntityTable( + override val mapping: ElementMapping, + override val table: DataFrameTable +) extends ElementTable[DataFrameTable] with RecordBehaviour { + + override type Records = SparkEntityTable + + private[spark] def records(implicit cypherEngine: SparkCypherSession): SparkCypherRecords = cypherEngine.records.fromElementTable(entityTable = this) + + override def cache(): SparkEntityTable = { + table.cache() + this + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala new file mode 100644 index 0000000000000..aea08f0a80831 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala @@ -0,0 +1,83 @@ +/* + * 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.spark.cypher + +import org.apache.hadoop.fs.Path + +object SparkGraphDirectoryStructure { + + import org.apache.spark.cypher.conversions.StringEncodingUtilities._ + + private implicit class StringPath(val path: String) extends AnyVal { + def /(segment: String): String = s"$path$pathSeparator$segment" + } + + val pathSeparator: String = Path.SEPARATOR + + val nodeTablesDirectoryName = "nodes" + + val relationshipTablesDirectoryName = "relationships" + + // Because an empty path does not work, we need a special directory name for nodes without labels. + val noLabelNodeDirectoryName: String = "__NO_LABEL__" + + def nodeTableDirectoryName(labels: Set[String]): String = concatDirectoryNames(labels.toSeq.sorted) + + def relKeyTableDirectoryName(relKey: String): String = relKey.encodeSpecialCharacters + + def concatDirectoryNames(seq: Seq[String]): String = { + if (seq.isEmpty) { + noLabelNodeDirectoryName + } else { + // TODO: Find more elegant solution for encoding underline characters + seq.map(_.replace("_", "--UNDERLINE--")).mkString("_").encodeSpecialCharacters + } + } + + implicit class ComboDirectory(val directoryName: String) extends AnyVal { + + def toLabelCombo: Set[String] = { + directoryName match { + case `noLabelNodeDirectoryName` => Set.empty + case encodedLabelString => + val encodedLabels = encodedLabelString.split('_').toSet + // TODO: Find more elegant solution for decoding underline characters + encodedLabels.map(_.decodeSpecialCharacters.replace("--UNDERLINE--", "_")) + } + } + + def toRelationshipType: String = directoryName.decodeSpecialCharacters + + } + +} + +case class SparkGraphDirectoryStructure(rootPath: String) { + + import SparkGraphDirectoryStructure._ + + def pathToNodeTable(labels: Set[String]): String = pathToNodeDirectory / nodeTableDirectoryName(labels) + + def pathToRelationshipTable(relKey: String): String = pathToRelationshipDirectory / relKeyTableDirectoryName(relKey) + + def pathToNodeDirectory: String = rootPath / nodeTablesDirectoryName + + def pathToRelationshipDirectory: String = rootPath / relationshipTablesDirectoryName + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala new file mode 100644 index 0000000000000..4698572e00b69 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala @@ -0,0 +1,267 @@ +/* + * 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.spark.cypher + +import org.apache.spark.cypher.conversions.ExprConversions._ +import org.apache.spark.cypher.conversions.TypeConversions._ +import org.apache.spark.sql.types.StructField +import org.apache.spark.sql._ +import org.apache.spark.storage.StorageLevel +import org.opencypher.okapi.api.types.CypherType +import org.opencypher.okapi.api.value.CypherValue +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherValue} +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, UnsupportedOperationException} +import org.opencypher.okapi.ir.api.expr._ +import org.opencypher.okapi.relational.api.table.Table +import org.opencypher.okapi.relational.impl.planning._ +import org.opencypher.okapi.relational.impl.table.RecordHeader + +import scala.collection.JavaConverters._ + +object SparkTable { + + implicit class DataFrameTable(val df: Dataset[Row]) extends Table[DataFrameTable] { + + private case class EmptyRow() + + override def physicalColumns: Seq[String] = df.columns + + override def columnType: Map[String, CypherType] = physicalColumns.map(c => c -> df.cypherTypeForColumn(c)).toMap + + override def rows: Iterator[String => CypherValue] = df.toLocalIterator().asScala.map { row => + physicalColumns.map(c => c -> CypherValue(row.get(row.fieldIndex(c)))).toMap + } + + override def size: Long = df.count() + + override def select(col: (String, String), cols: (String, String)*): DataFrameTable = { + val columns = col +: cols + if (df.columns.toSeq == columns.map { case (_, alias) => alias }) { + df + } else { + // Spark interprets dots in column names as struct accessors. Hence, we need to escape column names by default. + df.select(columns.map { case (colName, alias) => df.col(s"`$colName`").as(alias) }: _*) + } + } + + override def filter(expr: Expr)(implicit header: RecordHeader, parameters: CypherMap): DataFrameTable = { + df.where(expr.asSparkSQLExpr(header, df, parameters)) + } + + override def withColumns(columns: (Expr, String)*) + (implicit header: RecordHeader, parameters: CypherMap): DataFrameTable = { + val initialColumnNameToColumn: Map[String, Column] = df.columns.map(c => c -> df.col(c)).toMap + val updatedColumns = columns.foldLeft(initialColumnNameToColumn) { case (columnMap, (expr, columnName)) => + val column = expr.asSparkSQLExpr(header, df, parameters).as(columnName) + columnMap + (columnName -> column) + } + // TODO: Re-enable this check as soon as types (and their nullability) are correctly inferred in typing phase + // if (!expr.cypherType.isNullable) { + // withColumn.setNonNullable(column) + // } else { + // withColumn + // } + val existingColumnNames = df.columns + // Preserve order of existing columns + val columnsForSelect = existingColumnNames.map(updatedColumns) ++ + updatedColumns.filterKeys(!existingColumnNames.contains(_)).values + + df.select(columnsForSelect: _*) + } + + override def drop(cols: String*): DataFrameTable = { + df.drop(cols: _*) + } + + override def orderBy(sortItems: (Expr, Order)*) + (implicit header: RecordHeader, parameters: CypherMap): DataFrameTable = { + val mappedSortItems = sortItems.map { case (expr, order) => + val mappedExpr = expr.asSparkSQLExpr(header, df, parameters) + order match { + case Ascending => mappedExpr.asc + case Descending => mappedExpr.desc + } + } + df.orderBy(mappedSortItems: _*) + } + + override def skip(items: Long): DataFrameTable = { + // TODO: Replace with data frame based implementation ASAP + df.sparkSession.createDataFrame( + df.rdd + .zipWithIndex() + .filter(pair => pair._2 >= items) + .map(_._1), + df.toDF().schema + ) + } + + override def limit(items: Long): DataFrameTable = { + if (items > Int.MaxValue) throw IllegalArgumentException("an integer", items) + df.limit(items.toInt) + } + + override def group(by: Set[Var], aggregations: Map[String, Aggregator]) + (implicit header: RecordHeader, parameters: CypherMap): DataFrameTable = { + + def withInnerExpr(expr: Expr)(f: Column => Column) = + f(expr.asSparkSQLExpr(header, df, parameters)) + + val data: Either[RelationalGroupedDataset, Dataset[Row]] = + if (by.nonEmpty) { + val columns = by.flatMap { expr => + val withChildren = header.ownedBy(expr) + withChildren.map(e => withInnerExpr(e)(identity)) + } + Left(df.groupBy(columns.toSeq: _*)) + } else { + Right(df) + } + + val sparkAggFunctions = aggregations.map { + case (columnName, aggFunc) => aggFunc.asSparkSQLExpr(header, df, parameters).as(columnName) + } + + data.fold( + _.agg(sparkAggFunctions.head, sparkAggFunctions.tail.toSeq: _*), + _.agg(sparkAggFunctions.head, sparkAggFunctions.tail.toSeq: _*) + ) + } + + override def unionAll(other: DataFrameTable): DataFrameTable = { + val leftTypes = df.schema.fields.flatMap(_.toCypherType) + val rightTypes = other.df.schema.fields.flatMap(_.toCypherType) + + leftTypes.zip(rightTypes).foreach { + case (leftType, rightType) if !leftType.nullable.couldBeSameTypeAs(rightType.nullable) => + throw IllegalArgumentException( + "Equal column data types for union all (differing nullability is OK)", + s"Left fields: ${df.schema.fields.mkString(", ")}\n\tRight fields: ${other.df.schema.fields.mkString(", ")}") + case _ => + } + + df.union(other.df) + } + + override def join(other: DataFrameTable, joinType: JoinType, joinCols: (String, String)*): DataFrameTable = { + val joinTypeString = joinType match { + case InnerJoin => "inner" + case LeftOuterJoin => "left_outer" + case RightOuterJoin => "right_outer" + case FullOuterJoin => "full_outer" + case CrossJoin => "cross" + } + + joinType match { + case CrossJoin => + df.crossJoin(other.df) + + case LeftOuterJoin + if joinCols.isEmpty && df.sparkSession.conf.get("spark.sql.crossJoin.enabled", "false") == "false" => + throw UnsupportedOperationException("OPTIONAL MATCH support requires spark.sql.crossJoin.enabled=true") + + case _ => + df.safeJoin(other.df, joinCols, joinTypeString) + } + } + + override def distinct: DataFrameTable = distinct(df.columns: _*) + + override def distinct(colNames: String*): DataFrameTable = { + df.dropDuplicates(colNames) + } + + override def cache(): DataFrameTable = { + val planToCache = df.queryExecution.analyzed + if (df.sparkSession.sharedState.cacheManager.lookupCachedData(planToCache).nonEmpty) { + df.sparkSession.sharedState.cacheManager.cacheQuery(df, None, StorageLevel.MEMORY_ONLY) + } + this + } + + override def show(rows: Int): Unit = df.show(rows) + + def persist(): DataFrameTable = df.persist() + + def persist(newLevel: StorageLevel): DataFrameTable = df.persist(newLevel) + + def unpersist(): DataFrameTable = df.unpersist() + + def unpersist(blocking: Boolean): DataFrameTable = df.unpersist(blocking) + } + + implicit class DataFrameOps(val df: Dataset[Row]) extends AnyVal { + + def safeJoin(other: Dataset[Row], joinCols: Seq[(String, String)], joinType: String): Dataset[Row] = { + require(joinCols.map(_._1).forall(col => !other.columns.contains(col))) + require(joinCols.map(_._2).forall(col => !df.columns.contains(col))) + + val joinExpr = if (joinCols.nonEmpty) { + joinCols.map { + case (l, r) => df.col(l) === other.col(r) + }.reduce((acc, expr) => acc && expr) + } else { + functions.lit(true) + } + df.join(other, joinExpr, joinType) + } + + def safeDropColumns(names: String*): Dataset[Row] = { + val nonExistentColumns = names.toSet -- df.columns + require(nonExistentColumns.isEmpty, + s"Cannot drop column(s) ${nonExistentColumns.map(c => s"`$c`").mkString(", ")}. They do not exist.") + df.drop(names: _*) + } + + def cypherTypeForColumn(columnName: String): CypherType = { + val structField = structFieldForColumn(columnName) + val compatibleCypherType = structField.dataType.cypherCompatibleDataType.flatMap(_.toCypherType(structField.nullable)) + compatibleCypherType.getOrElse( + throw IllegalArgumentException("a supported Spark DataType that can be converted to CypherType", structField.dataType)) + } + + def structFieldForColumn(columnName: String): StructField = { + if (df.schema.fieldIndex(columnName) < 0) { + throw IllegalArgumentException(s"column with name $columnName", s"columns with names ${df.columns.mkString("[", ", ", "]")}") + } + df.schema.fields(df.schema.fieldIndex(columnName)) + } + + def safeRenameColumns(renames: (String, String)*): Dataset[Row] = { + safeRenameColumns(renames.toMap) + } + + def safeRenameColumns(renames: Map[String, String]): Dataset[Row] = { + if (renames.isEmpty || renames.forall { case (oldColumn, newColumn) => oldColumn == newColumn }) { + df + } else { + renames.foreach { case (oldName, newName) => require(!df.columns.contains(newName), + s"Cannot rename column `$oldName` to `$newName`. A column with name `$newName` exists already.") + } + val newColumns = df.columns.map { + case col if renames.contains(col) => renames(col) + case col => col + } + df.toDF(newColumns: _*) + } + } + + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/MappingAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/MappingAdapter.scala new file mode 100644 index 0000000000000..d2eea2b06c013 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/MappingAdapter.scala @@ -0,0 +1,43 @@ +/* + * 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.spark.cypher.adapters + +import org.apache.spark.graph.api.{NodeDataset, RelationshipDataset} +import org.opencypher.okapi.api.io.conversion.{ElementMapping, NodeMappingBuilder, RelationshipMappingBuilder} + +object MappingAdapter { + + implicit class RichNodeDataDataset(val nodeDf: NodeDataset) extends AnyVal { + def toNodeMapping: ElementMapping = NodeMappingBuilder + .on(nodeDf.idColumn) + .withImpliedLabels(nodeDf.labelSet.toSeq: _*) + .withPropertyKeyMappings(nodeDf.propertyColumns.toSeq:_*) + .build + } + + implicit class RichRelationshipDataDataset(val relDf: RelationshipDataset) extends AnyVal { + def toRelationshipMapping: ElementMapping = RelationshipMappingBuilder + .on(relDf.idColumn) + .withSourceStartNodeKey(relDf.sourceIdColumn) + .withSourceEndNodeKey(relDf.targetIdColumn) + .withRelType(relDf.relationshipType) + .withPropertyKeyMappings(relDf.propertyColumns.toSeq: _*) + .build + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/RelationalGraphAdapter.scala new file mode 100644 index 0000000000000..8e36c52346709 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/RelationalGraphAdapter.scala @@ -0,0 +1,94 @@ +/* + * 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.spark.cypher.adapters + +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.adapters.MappingAdapter._ +import org.apache.spark.cypher.io.SparkCypherPropertyGraphWriter +import org.apache.spark.cypher.{SparkCypherSession, SparkEntityTable} +import org.apache.spark.graph.api.{NodeDataset, PropertyGraph, PropertyGraphSchema, RelationshipDataset} +import org.apache.spark.sql.{Dataset, Row} +import org.opencypher.okapi.api.types.{CTNode, CTRelationship} +import org.opencypher.okapi.ir.api.expr.Var + +case class RelationalGraphAdapter( + cypherSession: SparkCypherSession, + nodeFrames: Seq[NodeDataset], + relationshipFrames: Seq[RelationshipDataset]) extends PropertyGraph { + + override def schema: PropertyGraphSchema = SchemaAdapter(graph.schema) + + private[cypher] lazy val graph = { + if (nodeFrames.isEmpty) { + cypherSession.graphs.empty + } else { + val nodeTables = nodeFrames.map { nodeDS => SparkEntityTable(nodeDS.toNodeMapping, nodeDS.ds) } + val relTables = relationshipFrames.map { relDS => SparkEntityTable(relDS.toRelationshipMapping, relDS.ds) } + cypherSession.graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*) + } + } + + private lazy val _nodeFrame: Map[Set[String], NodeDataset] = nodeFrames.map(nf => nf.labelSet -> nf).toMap + + private lazy val _relationshipFrame: Map[String, RelationshipDataset] = relationshipFrames.map(rf => rf.relationshipType -> rf).toMap + + override def nodes: Dataset[Row] = { + // TODO: move to API as default implementation + val nodeVar = Var("n")(CTNode) + val nodes = graph.nodes(nodeVar.name) + + val df = nodes.table.df + val header = nodes.header + + val idRename = header.column(nodeVar) -> "$ID" + val labelRenames = header.labelsFor(nodeVar).map(hasLabel => header.column(hasLabel) -> s":${hasLabel.label.name}").toSeq.sortBy(_._2) + val propertyRenames = header.propertiesFor(nodeVar).map(property => header.column(property) -> property.key.name).toSeq.sortBy(_._2) + + val selectColumns = (Seq(idRename) ++ labelRenames ++ propertyRenames).map { case (oldColumn, newColumn) => df.col(oldColumn).as(newColumn) } + + df.select(selectColumns: _*) + } + + override def relationships: Dataset[Row] = { + // TODO: move to API as default implementation + val relVar = Var("r")(CTRelationship) + val rels = graph.relationships(relVar.name) + + val df = rels.table.df + val header = rels.header + + val idRename = header.column(relVar) -> "$ID" + val sourceIdRename = header.column(header.startNodeFor(relVar)) -> "$SOURCE_ID" + val targetIdRename = header.column(header.endNodeFor(relVar)) -> "$TARGET_ID" + val relTypeRenames = header.typesFor(relVar).map(hasType => header.column(hasType) -> s":${hasType.relType.name}").toSeq.sortBy(_._2) + val propertyRenames = header.propertiesFor(relVar).map(property => header.column(property) -> property.key.name).toSeq.sortBy(_._2) + + val selectColumns = (Seq(idRename, sourceIdRename, targetIdRename) ++ relTypeRenames ++ propertyRenames).map { case (oldColumn, newColumn) => df.col(oldColumn).as(newColumn) } + + df.select(selectColumns: _*) + } + + override def nodeDataset(labelSet: Array[String]): NodeDataset = _nodeFrame(labelSet.toSet) + + override def relationshipDataset(relationshipType: String): RelationshipDataset = _relationshipFrame(relationshipType) + + override def write(): SparkCypherPropertyGraphWriter = + + new SparkCypherPropertyGraphWriter(this) +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/SchemaAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/SchemaAdapter.scala new file mode 100644 index 0000000000000..0a65d1135aed9 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/SchemaAdapter.scala @@ -0,0 +1,31 @@ +/* + * 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.spark.cypher.adapters + +import org.apache.spark.graph.api.PropertyGraphSchema +import org.opencypher.okapi.api.schema.{PropertyGraphSchema => OKAPISchema} + +case class SchemaAdapter(schema: OKAPISchema) extends PropertyGraphSchema { + + override def labelSets: Array[Array[String]] = schema.labelCombinations.combos.map(_.toArray).toArray + + override def relationshipTypes: Array[String] = schema.relationshipTypes.toArray + + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/CypherValueEncoders.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/CypherValueEncoders.scala new file mode 100644 index 0000000000000..21e3aed6bd690 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/CypherValueEncoders.scala @@ -0,0 +1,54 @@ +/* + * 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.spark.cypher.conversions + +import org.apache.spark.cypher.{SparkCypherNode, SparkCypherRelationship} +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.Encoders.kryo +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherValue} + +import scala.language.implicitConversions + +object CypherValueEncoders { + + private implicit def asExpressionEncoder[T](v: Encoder[T]): ExpressionEncoder[T] = { + v.asInstanceOf[ExpressionEncoder[T]] + } + + implicit def cypherValueEncoder: ExpressionEncoder[CypherValue] = { + kryo[CypherValue] + } + + implicit def cypherRecordEncoder: ExpressionEncoder[Map[String, CypherValue]] = { + kryo[Map[String, CypherValue]] + } + + implicit def cypherNodeEncoder: ExpressionEncoder[SparkCypherNode] = { + kryo[SparkCypherNode] + } + + implicit def cypherRelationshipEncoder: ExpressionEncoder[SparkCypherRelationship] = { + kryo[SparkCypherRelationship] + } + + implicit def cypherMapEncoder: ExpressionEncoder[CypherMap] = { + kryo[CypherMap] + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/ExprConversions.scala new file mode 100644 index 0000000000000..6a772bb0f0245 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/ExprConversions.scala @@ -0,0 +1,347 @@ +/* + * 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.spark.cypher.conversions + +import org.apache.spark.cypher.SparkCypherFunctions._ +import org.apache.spark.cypher.conversions.TemporalConversions._ +import org.apache.spark.cypher.conversions.TypeConversions._ +import org.apache.spark.cypher.udfs.TemporalUdfs +import org.apache.spark.sql.catalyst.expressions.CaseWhen +import org.apache.spark.sql.functions.{array_contains => _, translate => _, _} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Column, Dataset, Row} +import org.opencypher.okapi.api.types._ +import org.opencypher.okapi.api.value.CypherValue.CypherMap +import org.opencypher.okapi.impl.exception._ +import org.opencypher.okapi.ir.api.expr._ +import org.opencypher.okapi.relational.impl.table.RecordHeader + +object ExprConversions { + + implicit class RichExpression(expr: Expr) { + + /** + * Converts `expr` with the `withConvertedChildren` function, which is passed the converted child expressions as its + * argument. + * + * Iff the expression has `expr.nullInNullOut == true`, then any child being mapped to `null` will also result in + * the parent expression being mapped to null. + * + * For these expressions the `withConvertedChildren` function is guaranteed to not receive any `null` + * values from the evaluated children. + */ + def nullSafeConversion(expr: Expr)(withConvertedChildren: Seq[Column] => Column) + (implicit header: RecordHeader, ds: Dataset[Row], parameters: CypherMap): Column = { + if (expr.cypherType == CTNull) { + NULL_LIT + } else if (expr.cypherType == CTTrue) { + TRUE_LIT + } else if (expr.cypherType == CTFalse) { + FALSE_LIT + } else { + val evaluatedArgs = expr.children.map(_.asSparkSQLExpr) + val withConvertedChildrenResult = withConvertedChildren(evaluatedArgs).expr + if (expr.children.nonEmpty && expr.nullInNullOut && expr.cypherType.isNullable) { + val nullPropagationCases = evaluatedArgs.map(_.isNull.expr).zip(Seq.fill(evaluatedArgs.length)(NULL_LIT.expr)) + new Column(CaseWhen(nullPropagationCases, withConvertedChildrenResult)) + } else { + new Column(withConvertedChildrenResult) + } + } + } + + /** + * Attempts to create a Spark SQL expression from the CAPS expression. + * + * @param header the header of the CAPSRecords in which the expression should be evaluated. + * @param ds the dataset containing the data over which the expression should be evaluated. + * @param parameters query parameters + * @return Some Spark SQL expression if the input was mappable, otherwise None. + */ + def asSparkSQLExpr(implicit header: RecordHeader, ds: Dataset[Row], parameters: CypherMap): Column = { + val outCol = expr match { + // Evaluate based on already present data; no recursion + case _: Var | _: HasLabel | _: HasType | _: StartNode | _: EndNode => column_for(expr) + // Evaluate bottom-up + case _ => nullSafeConversion(expr)(convert) + } + header.getColumn(expr) match { + case None => outCol + case Some(colName) => outCol.as(colName) + } + } + + private def convert(convertedChildren: Seq[Column]) + (implicit header: RecordHeader, ds: Dataset[Row], parameters: CypherMap): Column = { + + def child0: Column = convertedChildren.head + + def child1: Column = convertedChildren(1) + + def child2: Column = convertedChildren(2) + + expr match { + case _: ListLit => array(convertedChildren: _*) + case l: Lit[_] => lit(l.v) + case _: AliasExpr => child0 + case Param(name) => parameters(name).toSparkLiteral + + // Predicates + case _: Equals => child0 === child1 + case _: Not => !child0 + case Size(e) => { + e.cypherType match { + case CTString => length(child0) + case _ => size(child0) // it's a list + } + }.cast(LongType) + case _: Ands => convertedChildren.foldLeft(TRUE_LIT)(_ && _) + case _: Ors => convertedChildren.foldLeft(FALSE_LIT)(_ || _) + case _: IsNull => child0.isNull + case _: IsNotNull => child0.isNotNull + case _: Exists => child0.isNotNull + case _: LessThan => child0 < child1 + case _: LessThanOrEqual => child0 <= child1 + case _: GreaterThanOrEqual => child0 >= child1 + case _: GreaterThan => child0 > child1 + + case _: StartsWith => child0.startsWith(child1) + case _: EndsWith => child0.endsWith(child1) + case _: Contains => child0.contains(child1) + case _: RegexMatch => regex_match(child0, child1) + + // Other + case Explode(list) => list.cypherType match { + case CTNull => explode(NULL_LIT.cast(ArrayType(NullType))) + case _ => explode(child0) + } + + case _: ElementProperty => if (!header.contains(expr)) NULL_LIT else column_for(expr) + case MapProperty(_, key) => if (expr.cypherType.material == CTVoid) NULL_LIT else child0.getField(key.name) + case DateProperty(_, key) => temporalAccessor[java.sql.Date](child0, key.name) + case LocalDateTimeProperty(_, key) => temporalAccessor[java.sql.Timestamp](child0, key.name) + case DurationProperty(_, key) => TemporalUdfs.durationAccessor(key.name.toLowerCase).apply(child0) + + case LocalDateTime(maybeDateExpr) => maybeDateExpr.map(e => lit(e.resolveTimestamp).cast(DataTypes.TimestampType)).getOrElse(current_timestamp()) + case Date(maybeDateExpr) => maybeDateExpr.map(e => lit(e.resolveDate).cast(DataTypes.DateType)).getOrElse(current_timestamp()) + case Duration(durationExpr) => lit(durationExpr.resolveInterval) + + case In(lhs, rhs) => rhs.cypherType.material match { + case CTList(inner) if inner.couldBeSameTypeAs(lhs.cypherType) => array_contains(child1, child0) + case _ => NULL_LIT + } + + // Arithmetic + case Add(lhs, rhs) => + val lhsCT = lhs.cypherType.material + val rhsCT = rhs.cypherType.material + lhsCT -> rhsCT match { + case (CTList(lhInner), CTList(rhInner)) => + if ((lhInner | rhInner).isSparkCompatible) { + concat(child0, child1) + } else { + throw NotImplementedException(s"Lists of different inner types are not supported (${lhInner.material}, ${rhInner.material})") + } + case (CTList(inner), nonListType) if (inner | nonListType).isSparkCompatible => concat(child0, array(child1)) + case (nonListType, CTList(inner)) if (inner | nonListType).isSparkCompatible => concat(array(child0), child1) + case (CTString, _) if rhsCT.subTypeOf(CTNumber) => concat(child0, child1.cast(StringType)) + case (_, CTString) if lhsCT.subTypeOf(CTNumber) => concat(child0.cast(StringType), child1) + case (CTString, CTString) => concat(child0, child1) + case (CTDate, CTDuration) => TemporalUdfs.dateAdd(child0, child1) + case _ => child0 + child1 + } + + case Subtract(lhs, rhs) if lhs.cypherType.material.subTypeOf(CTDate) && rhs.cypherType.material.subTypeOf(CTDuration) => + TemporalUdfs.dateSubtract(child0, child1) + + case _: Subtract => child0 - child1 + + case _: Multiply => child0 * child1 + case div: Divide => (child0 / child1).cast(div.cypherType.getSparkType) + + // Id functions + case _: Id => child0 + + // Functions + case _: MonotonicallyIncreasingId => monotonically_increasing_id() + case Labels(e) => + val possibleLabels = header.labelsFor(e.owner.get).toSeq.sortBy(_.label.name) + val labelBooleanFlagsCol = possibleLabels.map(_.asSparkSQLExpr) + val nodeLabels = filter_true(possibleLabels.map(_.label.name), labelBooleanFlagsCol) + nodeLabels + + case Type(e) => + val possibleRelTypes = header.typesFor(e.owner.get).toSeq.sortBy(_.relType.name) + val relTypeBooleanFlagsCol = possibleRelTypes.map(_.asSparkSQLExpr) + val relTypes = filter_true(possibleRelTypes.map(_.relType.name), relTypeBooleanFlagsCol) + val relType = get_array_item(relTypes, index = 0) + relType + + case Keys(e) => + e.cypherType.material match { + case entity if entity.subTypeOf(CTElement) => + val possibleProperties = header.propertiesFor(e.owner.get).toSeq.sortBy(_.key.name) + val propertyNames = possibleProperties.map(_.key.name) + val propertyValues = possibleProperties.map(_.asSparkSQLExpr) + filter_not_null(propertyNames, propertyValues) + + case CTMap(inner) => + val mapColumn = child0 + val (propertyKeys, propertyValues) = inner.keys.map { e => + // Whe have to make sure that every column has the same type (true or null) + e -> when(mapColumn.getField(e).isNotNull, TRUE_LIT).otherwise(NULL_LIT) + }.toSeq.unzip + filter_not_null(propertyKeys, propertyValues) + + case other => throw IllegalArgumentException("an Expression with type CTNode, CTRelationship or CTMap", other) + } + + case Properties(e) => + e.cypherType.material match { + case entity if entity.subTypeOf(CTElement) => + val propertyExpressions = header.propertiesFor(e.owner.get).toSeq.sortBy(_.key.name) + val propertyColumns = propertyExpressions + .map(propertyExpression => propertyExpression.asSparkSQLExpr.as(propertyExpression.key.name)) + create_struct(propertyColumns) + case _: CTMap => child0 + case other => + throw IllegalArgumentException("a node, relationship or map", other, "Invalid input to properties function") + } + + case StartNodeFunction(e) => header.startNodeFor(e.owner.get).asSparkSQLExpr + case EndNodeFunction(e) => header.endNodeFor(e.owner.get).asSparkSQLExpr + + case _: ToFloat => child0.cast(DoubleType) + case _: ToInteger => child0.cast(IntegerType) + case _: ToString => child0.cast(StringType) + case _: ToBoolean => child0.cast(BooleanType) + + case _: Trim => trim(child0) + case _: LTrim => ltrim(child0) + case _: RTrim => rtrim(child0) + case _: ToUpper => upper(child0) + case _: ToLower => lower(child0) + + case _: Range => sequence(child0, child1, convertedChildren.lift(2).getOrElse(ONE_LIT)) + + case _: Replace => translate(child0, child1, child2) + + case _: Substring => child0.substr(child1 + ONE_LIT, convertedChildren.lift(2).getOrElse(length(child0) - child1)) + + // Mathematical functions + case E => E_LIT + case Pi => PI_LIT + + case _: Sqrt => sqrt(child0) + case _: Log => log(child0) + case _: Log10 => log(10.0, child0) + case _: Exp => exp(child0) + case _: Abs => abs(child0) + case _: Ceil => ceil(child0).cast(DoubleType) + case _: Floor => floor(child0).cast(DoubleType) + case Rand => rand() + case _: Round => round(child0).cast(DoubleType) + case _: Sign => signum(child0).cast(IntegerType) + + case _: Acos => acos(child0) + case _: Asin => asin(child0) + case _: Atan => atan(child0) + case _: Atan2 => atan2(child0, child1) + case _: Cos => cos(child0) + case Cot(e) => Divide(IntegerLit(1), Tan(e)).asSparkSQLExpr + case _: Degrees => degrees(child0) + case Haversin(e) => Divide(Subtract(IntegerLit(1), Cos(e)), IntegerLit(2)).asSparkSQLExpr + case _: Radians => radians(child0) + case _: Sin => sin(child0) + case _: Tan => tan(child0) + + // Time functions + case Timestamp => current_timestamp().cast(LongType) + + // Bit operations + case _: BitwiseAnd => child0.bitwiseAND(child1) + case _: BitwiseOr => child0.bitwiseOR(child1) + case ShiftLeft(_, IntegerLit(shiftBits)) => shiftLeft(child0, shiftBits.toInt) + case ShiftRightUnsigned(_, IntegerLit(shiftBits)) => shiftRightUnsigned(child0, shiftBits.toInt) + + // Pattern Predicate + case ep: ExistsPatternExpr => ep.targetField.asSparkSQLExpr + + case Coalesce(es) => + val columns = es.map(_.asSparkSQLExpr) + coalesce(columns: _*) + + case CaseExpr(_, maybeDefault) => + val (maybeConvertedDefault, convertedAlternatives) = if (maybeDefault.isDefined) { + Some(convertedChildren.head) -> convertedChildren.tail + } else { + None -> convertedChildren + } + val indexed = convertedAlternatives.zipWithIndex + val conditions = indexed.collect { case (c, i) if i % 2 == 0 => c } + val values = indexed.collect { case (c, i) if i % 2 == 1 => c } + val branches = conditions.zip(values) + switch(branches, maybeConvertedDefault) + + case ContainerIndex(container, index) => + val containerCol = container.asSparkSQLExpr + container.cypherType.material match { + case c if c.subTypeOf(CTContainer) => containerCol.get(index.asSparkSQLExpr) + case other => throw NotImplementedException(s"Accessing $other by index is not supported") + } + + case _: ListSliceFromTo => list_slice(child0, Some(child1), Some(child2)) + case _: ListSliceFrom => list_slice(child0, Some(child1), None) + case _: ListSliceTo => list_slice(child0, None, Some(child1)) + + case MapExpression(items) => expr.cypherType.material match { + case CTMap(_) => + val innerColumns = items.map { + case (key, innerExpr) => innerExpr.asSparkSQLExpr.as(key) + }.toSeq + create_struct(innerColumns) + case other => throw IllegalArgumentException("an expression of type CTMap", other) + } + + // Aggregators + case Count(_, distinct) => + if (distinct) countDistinct(child0) + else count(child0) + + case Collect(_, distinct) => + if (distinct) collect_set(child0) + else collect_list(child0) + + case CountStar => count(ONE_LIT) + case _: Avg => avg(child0) + case _: Max => max(child0) + case _: Min => min(child0) + case _: Sum => sum(child0) + + case BigDecimal(_, precision, scale) => + make_big_decimal(child0, precision.toInt, scale.toInt) + + case _ => + throw NotImplementedException(s"No support for converting Cypher expression $expr to a Spark SQL expression") + } + } + + } +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/GraphElementFrameConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/GraphElementFrameConversions.scala new file mode 100644 index 0000000000000..99c69fe147c4e --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/GraphElementFrameConversions.scala @@ -0,0 +1,57 @@ +/* + * 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.spark.cypher.conversions + +import org.apache.spark.graph.api.GraphElementDataset +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Dataset, Row} + +object GraphElementFrameConversions { + + def normalizeDf(frame: GraphElementDataset): Dataset[Row] = { + val mappedColumnNames = frame.idColumns.toSeq ++ frame.propertyColumns.values.toSeq.sorted + val mappedDf = if (mappedColumnNames == frame.ds.columns.toSeq) { + frame.ds + } else { + frame.ds.select(mappedColumnNames.map(frame.ds.col): _*) + } + if (frame.idColumns.forall(idColumn => frame.ds.schema(idColumn).dataType == BinaryType)) { + mappedDf + } else { + encodeIdColumns(mappedDf, frame.idColumns: _*) + } + } + + private def encodeIdColumns(ds: Dataset[Row], idColumnNames: String*): Dataset[Row] = { + val encodedIdCols = idColumnNames.map { idColumnName => + val col = ds.col(idColumnName) + ds.schema(idColumnName).dataType match { + case BinaryType => col + case StringType | ByteType | ShortType | IntegerType | LongType => col.cast(BinaryType) + // TODO: Constrain to types that make sense as IDs + case _ => col.cast(StringType).cast(BinaryType) + } + } + val remainingColumnNames = ds.columns.filterNot(idColumnNames.contains) + val remainingCols = remainingColumnNames.map(ds.col) + ds.select(encodedIdCols ++ remainingCols: _*) + } + +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala new file mode 100644 index 0000000000000..8cdfda7bf510c --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala @@ -0,0 +1,139 @@ +/* + * 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.spark.cypher.conversions + +import org.apache.spark.cypher.{SparkCypherNode, SparkCypherRelationship} +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.opencypher.okapi.api.types.{CTList, CTMap, CTNode, CTRelationship} +import org.opencypher.okapi.api.value.CypherValue._ +import org.opencypher.okapi.api.value._ +import org.opencypher.okapi.impl.exception.UnsupportedOperationException +import org.opencypher.okapi.ir.api.expr.{Expr, ListSegment, Var} +import org.opencypher.okapi.relational.impl.table.RecordHeader + +// TODO: argument cannot be a Map due to Scala issue https://issues.scala-lang.org/browse/SI-7005 +final case class RowConversion(exprToColumn: Seq[(Expr, String)]) extends (Row => CypherMap) { + + private val header = RecordHeader(exprToColumn.toMap) + + override def apply(row: Row): CypherMap = { + val values = header.returnItems.map(r => r.name -> constructValue(row, r)).toSeq + CypherMap(values: _*) + } + + // TODO: Validate all column types. At the moment null values are cast to the expected type... + private def constructValue(row: Row, v: Var): CypherValue = { + v.cypherType.material match { + case n if n.subTypeOf(CTNode.nullable) => collectNode(row, v) + case r if r.subTypeOf(CTRelationship.nullable) => collectRel(row, v) + case l if l.subTypeOf(CTList.nullable) && !header.exprToColumn.contains(v) => collectComplexList(row, v) + case _ => constructFromExpression(row, v) + } + } + + private def constructFromExpression(row: Row, expr: Expr): CypherValue = { + expr.cypherType.material match { + case CTMap(inner) => + if (inner.isEmpty) { + CypherMap() + } else { + val innerRow = row.getAs[GenericRowWithSchema](header.column(expr)) + innerRow match { + case _: GenericRowWithSchema => + innerRow.schema.fieldNames.map { field => + field -> CypherValue(innerRow.getAs[Any](field)) + }.toMap + case null => null + } + } + + case _ => + val raw = row.getAs[Any](header.column(expr)) + CypherValue(raw) + } + } + + private def collectNode(row: Row, v: Var): CypherValue = { + val idValue = row.getAs[Any](header.column(v)) + idValue match { + case null => CypherNull + case id: Array[_] => + + val labels = header + .labelsFor(v) + .map { l => l.label.name -> row.getAs[Boolean](header.column(l)) } + .collect { case (name, true) => name } + + val properties = header + .propertiesFor(v) + .map { p => p.key.name -> constructFromExpression(row, p) } + .collect { case (key, value) if !value.isNull => key -> value } + .toMap + + SparkCypherNode(id.asInstanceOf[Array[Byte]], labels, properties) + case invalidID => throw UnsupportedOperationException(s"CAPSNode ID has to be a CAPSId instead of ${invalidID.getClass}") + } + } + + private def collectRel(row: Row, v: Var): CypherValue = { + val idValue = row.getAs[Any](header.column(v)) + idValue match { + case null => CypherNull + case id: Array[_] => + val source = row.getAs[Array[_]](header.column(header.startNodeFor(v))) + val target = row.getAs[Array[_]](header.column(header.endNodeFor(v))) + + val relType = header + .typesFor(v) + .map { l => l.relType.name -> row.getAs[Boolean](header.column(l)) } + .collect { case (name, true) => name } + .head + + val properties = header + .propertiesFor(v) + .map { p => p.key.name -> constructFromExpression(row, p) } + .collect { case (key, value) if !value.isNull => key -> value } + .toMap + + SparkCypherRelationship( + id.asInstanceOf[Array[Byte]], + source.asInstanceOf[Array[Byte]], + target.asInstanceOf[Array[Byte]], + relType, + properties) + case invalidID => throw UnsupportedOperationException(s"CAPSRelationship ID has to be a Long instead of ${invalidID.getClass}") + } + } + + private def collectComplexList(row: Row, expr: Var): CypherList = { + val elements = header.ownedBy(expr).collect { + case p: ListSegment => p + }.toSeq.sortBy(_.index) + + val values = elements + .map(constructValue(row, _)) + .filter { + case CypherNull => false + case _ => true + } + + CypherList(values) + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/StringEncodingUtilities.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/StringEncodingUtilities.scala new file mode 100644 index 0000000000000..c837211ae5e07 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/StringEncodingUtilities.scala @@ -0,0 +1,121 @@ +/* + * 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.spark.cypher.conversions + +import scala.annotation.tailrec + +object StringEncodingUtilities { + + val propertyPrefix: String = "property_" + + val relTypePrefix: String = "relType_" + + protected val maxCharactersInHexStringEncoding: Int = 4 // Hex string encoding of a `Char` is up to 4 characters + + implicit class CharOps(val c: Char) extends AnyVal { + def isAscii: Boolean = c.toInt <= 127 + } + + implicit class StringOps(val s: String) extends AnyVal { + + def toPropertyColumnName: String = { + s"$propertyPrefix${s.encodeSpecialCharacters}" + } + + def isPropertyColumnName: Boolean = s.startsWith(propertyPrefix) + + def toProperty: String = { + if (s.isPropertyColumnName) { + s.drop(propertyPrefix.length).decodeSpecialCharacters + } else { + s + } + } + + def toRelTypeColumnName: String = { + s"$relTypePrefix${s.encodeSpecialCharacters}" + } + + def isRelTypeColumnName: Boolean = s.startsWith(relTypePrefix) + + /** + * Encodes special characters in a string. + * + * The encoded string contains only ASCII letters, numbers, '_', and '@'. The encoded string is compatible + * with both SQL column names and file paths. + * + * @return encoded string + */ + def encodeSpecialCharacters: String = { + val sb = new StringBuilder + + @tailrec def recEncode(index: Int): Unit = { + if (index < s.length) { + val charToEncode = s(index) + if (charToEncode == '_' || (charToEncode.isLetterOrDigit && charToEncode.isAscii)) { + sb.append(charToEncode) + } else { + sb.append("@") + val hexString = charToEncode.toHexString + // Pad left to max encoded length with '0's + for (_ <- 0 until maxCharactersInHexStringEncoding - hexString.length) sb.append('0') + sb.append(hexString) + } + recEncode(index + 1) + } + } + + recEncode(0) + sb.toString + } + + /** + * Recovers the original string from a string encoded with [[encodeSpecialCharacters]]. + * + * @return original string + */ + def decodeSpecialCharacters: String = { + val sb = new StringBuilder + + @tailrec def recDecode(index: Int): Unit = { + if (index < s.length) { + val charToDecode = s(index) + val nextIndex = if (charToDecode == '@') { + val encodedHexStringStart = index + 1 + val indexAfterHexStringEnd = encodedHexStringStart + maxCharactersInHexStringEncoding + val hexString = s.substring(encodedHexStringStart, indexAfterHexStringEnd) + sb.append(hexString.parseHex) + indexAfterHexStringEnd + } else { + sb.append(charToDecode) + index + 1 + } + recDecode(nextIndex) + } + } + + recDecode(0) + sb.toString + } + + def parseHex: Char = Integer.parseInt(s, 16).toChar + + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TemporalConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TemporalConversions.scala new file mode 100644 index 0000000000000..015b33b8d307a --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TemporalConversions.scala @@ -0,0 +1,168 @@ +/* + * 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.spark.cypher.conversions + +import java.sql.{Date, Timestamp} +import java.time.temporal.ChronoUnit + +import org.apache.spark.cypher.udfs.TemporalUdfs +import org.apache.spark.sql.{Column, functions} +import org.apache.spark.unsafe.types.CalendarInterval +import org.opencypher.okapi.api.value.CypherValue.{CypherInteger, CypherMap, CypherString} +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, IllegalStateException, NotImplementedException, UnsupportedOperationException} +import org.opencypher.okapi.impl.temporal.TemporalTypesHelper._ +import org.opencypher.okapi.impl.temporal.{Duration, TemporalConstants} +import org.opencypher.okapi.ir.api.expr.{Expr, MapExpression, NullLit, Param} +import org.opencypher.okapi.impl.temporal.{Duration => DurationValue} + +import scala.reflect.runtime.universe.TypeTag + +object TemporalConversions { + + implicit class RichDuration(duration: Duration) { + + /** + * Converts the Okapi representation of a duration into the spark representation. + * + * @note This conversion is lossy, as the Sparks [[CalendarInterval]] only has a resolution down to microseconds. + * Additionally it uses an approximate representation of days. + */ + def toCalendarInterval: CalendarInterval = { + if (duration.nanos % 1000 != 0) { + throw UnsupportedOperationException("Spark does not support durations with nanosecond resolution.") + } + + val microseconds = duration.nanos / 1000 + + duration.seconds * CalendarInterval.MICROS_PER_SECOND + + duration.days * CalendarInterval.MICROS_PER_DAY + + new CalendarInterval( + duration.months.toInt, + microseconds + ) + } + } + + /** + * Converts the Spark representation of a duration into the Okapi representation. + * + * @note To ensure compatibility with the reverse operation we estimate the number of days from the given seconds. + */ + implicit class RichCalendarInterval(calendarInterval: CalendarInterval) { + def toDuration: Duration = { + val seconds = calendarInterval.microseconds / CalendarInterval.MICROS_PER_SECOND + val normalizedDays = seconds / (CalendarInterval.MICROS_PER_DAY / CalendarInterval.MICROS_PER_SECOND) + val normalizedSeconds = seconds % (CalendarInterval.MICROS_PER_DAY / CalendarInterval.MICROS_PER_SECOND) + val normalizedNanos = calendarInterval.microseconds % CalendarInterval.MICROS_PER_SECOND * 1000 + + Duration(months = calendarInterval.months, + days = normalizedDays, + seconds = normalizedSeconds, + nanoseconds = normalizedNanos + ) + } + + def toJavaDuration: java.time.Duration = { + val micros = calendarInterval.microseconds + + (calendarInterval.months * TemporalConstants.AVG_DAYS_PER_MONTH * CalendarInterval.MICROS_PER_DAY).toLong + java.time.Duration.of(micros, ChronoUnit.MICROS) + } + } + + implicit class TemporalExpression(val expr: Expr) extends AnyVal { + + def resolveTimestamp(implicit parameters: CypherMap): Timestamp = { + expr.resolveTemporalArgument + .map(parseLocalDateTime) + .map(java.sql.Timestamp.valueOf) + .map { + case ts if ts.getNanos % 1000 == 0 => ts + case _ => throw IllegalStateException("Spark does not support nanosecond resolution in 'localdatetime'") + } + .orNull + } + + def resolveDate(implicit parameters: CypherMap): Date = { + expr.resolveTemporalArgument + .map(parseDate) + .map(java.sql.Date.valueOf) + .orNull + } + + def resolveInterval(implicit parameters: CypherMap): CalendarInterval = { + expr.resolveTemporalArgument.map { + case Left(m) => DurationValue(m.mapValues(_.toLong)).toCalendarInterval + case Right(s) => DurationValue.parse(s).toCalendarInterval + }.orNull + } + + def resolveTemporalArgument(implicit parameters: CypherMap): Option[Either[Map[String, Int], String]] = { + expr match { + case MapExpression(inner) => + val map = inner.map { + case (key, Param(name)) => key -> (parameters(name) match { + case CypherString(s) => s.toInt + case CypherInteger(i) => i.toInt + case other => throw IllegalArgumentException("A map value of type CypherString or CypherInteger", other) + }) + case (key, e) => + throw NotImplementedException(s"Parsing temporal values is currently only supported for Literal-Maps, got $key -> $e") + } + + Some(Left(map)) + + case Param(name) => + val s = parameters(name) match { + case CypherString(str) => str + case other => throw IllegalArgumentException(s"Parameter `$name` to be a CypherString", other) + } + + Some(Right(s)) + + case NullLit => None + + case other => + throw NotImplementedException(s"Parsing temporal values is currently only supported for Literal-Maps and String literals, got $other") + } + } + + } + + def temporalAccessor[I: TypeTag](temporalColumn: Column, accessor: String): Column = { + accessor.toLowerCase match { + case "year" => functions.year(temporalColumn) + case "quarter" => functions.quarter(temporalColumn) + case "month" => functions.month(temporalColumn) + case "week" => functions.weekofyear(temporalColumn) + case "day" => functions.dayofmonth(temporalColumn) + case "ordinalday" => functions.dayofyear(temporalColumn) + case "weekyear" => TemporalUdfs.weekYear[I].apply(temporalColumn) + case "dayofquarter" => TemporalUdfs.dayOfQuarter[I].apply(temporalColumn) + case "dayofweek" | "weekday" => TemporalUdfs.dayOfWeek[I].apply(temporalColumn) + + case "hour" => functions.hour(temporalColumn) + case "minute" => functions.minute(temporalColumn) + case "second" => functions.second(temporalColumn) + case "millisecond" => TemporalUdfs.milliseconds[I].apply(temporalColumn) + case "microsecond" => TemporalUdfs.microseconds[I].apply(temporalColumn) + case other => throw UnsupportedOperationException(s"Unknown Temporal Accessor: $other") + } + } +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TypeConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TypeConversions.scala new file mode 100644 index 0000000000000..6d157a4af6655 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TypeConversions.scala @@ -0,0 +1,200 @@ +/* + * 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.spark.cypher.conversions + +import org.apache.spark.cypher.conversions.TemporalConversions._ +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval +import org.opencypher.okapi.api.types._ +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherValue, CypherValueConverter} +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, NotImplementedException} +import org.opencypher.okapi.ir.api.expr.Var +import org.opencypher.okapi.relational.impl.table.RecordHeader + +object TypeConversions { + + val DEFAULT_PRECISION = 20 + + implicit class CypherTypeOps(val ct: CypherType) extends AnyVal { + + def toStructField(column: String): StructField = { + ct.toSparkType match { + case Some(st) => StructField(column, st, ct.isNullable) + case None => throw IllegalArgumentException("CypherType supported by CAPS", ct) + } + } + + def toSparkType: Option[DataType] = ct match { + case CTNull => Some(NullType) + case _ => + ct.material match { + case CTString => Some(StringType) + case CTInteger => Some(LongType) + case CTBigDecimal(p, s) => Some(DataTypes.createDecimalType(p, s)) + case CTFloat => Some(DoubleType) + case CTLocalDateTime => Some(TimestampType) + case CTDate => Some(DateType) + case CTDuration => Some(CalendarIntervalType) + case CTIdentity => Some(BinaryType) + case b if b.subTypeOf(CTBoolean) => Some(BooleanType) + case n if n.subTypeOf(CTElement.nullable) => Some(BinaryType) + // Spark uses String as the default array inner type + case CTMap(inner) => Some(StructType(inner.map { case (key, vType) => vType.toStructField(key) }.toSeq)) + case CTEmptyList => Some(ArrayType(StringType, containsNull = false)) + case CTList(CTNull) => Some(ArrayType(StringType, containsNull = true)) + case CTList(inner) if inner.subTypeOf(CTBoolean.nullable) => Some(ArrayType(BooleanType, containsNull = inner.isNullable)) + case CTList(elemType) if elemType.toSparkType.isDefined => elemType.toSparkType.map(ArrayType(_, elemType.isNullable)) + case l if l.subTypeOf(CTList(CTNumber.nullable)) => Some(ArrayType(DoubleType, containsNull = l.isNullable)) + case _ => None + } + } + + def getSparkType: DataType = toSparkType match { + case Some(t) => t + case None => throw NotImplementedException(s"Mapping of CypherType $ct to Spark type is unsupported") + } + + def isSparkCompatible: Boolean = toSparkType.isDefined + + def ensureSparkCompatible(): Unit = getSparkType + + } + + implicit class StructTypeOps(val structType: StructType) { + def toRecordHeader: RecordHeader = { + + val exprToColumn = structType.fields.map { field => + val cypherType = field.toCypherType match { + case Some(ct) => ct + case None => throw IllegalArgumentException("a supported Spark type", field.dataType) + } + Var(field.name)(cypherType) -> field.name + } + + RecordHeader(exprToColumn.toMap) + } + + def binaryColumns: Set[String] = structType.fields.filter(_.dataType == BinaryType).map(_.name).toSet + + def convertTypes(from: DataType, to: DataType): StructType = StructType(structType.map { + case sf: StructField if sf.dataType == from => sf.copy(dataType = to) + case sf: StructField => sf + }) + } + + implicit class StructFieldOps(val field: StructField) extends AnyVal { + def toCypherType: Option[CypherType] = field.dataType.toCypherType(field.nullable) + } + + implicit class DataTypeOps(val dt: DataType) extends AnyVal { + def toCypherType(nullable: Boolean = false): Option[CypherType] = { + val result = dt match { + case StringType => Some(CTString) + case IntegerType => Some(CTInteger) + case LongType => Some(CTInteger) + case BooleanType => Some(CTBoolean) + case DoubleType => Some(CTFloat) + case dt: DecimalType => Some(CTBigDecimal(dt.precision, dt.scale)) + case TimestampType => Some(CTLocalDateTime) + case DateType => Some(CTDate) + case CalendarIntervalType => Some(CTDuration) + case ArrayType(NullType, _) => Some(CTEmptyList) + case BinaryType => Some(CTIdentity) + case ArrayType(elemType, containsNull) => + elemType.toCypherType(containsNull).map(CTList(_)) + case NullType => Some(CTNull) + case StructType(fields) => + val convertedFields = fields.map { field => field.name -> field.dataType.toCypherType(field.nullable) }.toMap + val containsNone = convertedFields.exists { + case (_, None) => true + case _ => false + } + if (containsNone) None else Some(CTMap(convertedFields.mapValues(_.get))) + case _ => None + } + + if (nullable) result.map(_.nullable) else result.map(_.material) + } + + /** + * Checks if the given data type is supported within the Cypher type system. + * + * @return true, iff the data type is supported + */ + def isCypherCompatible: Boolean = cypherCompatibleDataType.isDefined + + /** + * Converts the given Spark data type into a Cypher type system compatible Spark data type. + * + * @return some Cypher-compatible Spark data type or none if not compatible + */ + def cypherCompatibleDataType: Option[DataType] = dt match { + case ByteType | ShortType | IntegerType => Some(LongType) + case FloatType => Some(DoubleType) + case compatible if dt.toCypherType().isDefined => Some(compatible) + case _ => None + } + } + + implicit class RecordHeaderOps(header: RecordHeader) extends Serializable { + + def toStructType: StructType = { + val structFields = header.columns.toSeq.sorted.map { column => + val expressions = header.expressionsFor(column) + val commonType = expressions.map(_.cypherType).reduce(_ join _) + assert(commonType.isSparkCompatible, + s""" + |Expressions $expressions with common super type $commonType mapped to column $column have no compatible data type. + """.stripMargin) + commonType.toStructField(column) + } + StructType(structFields) + } + + def rowEncoder: ExpressionEncoder[Row] = + RowEncoder(header.toStructType) + } + + implicit class RowOps(row: Row) { + + def allNull: Boolean = allNull(row.size) + + def allNull(rowSize: Int): Boolean = (for (i <- 0 until rowSize) yield row.isNullAt(i)).reduce(_ && _) + } + + + object SparkCypherValueConverter extends CypherValueConverter { + override def convert(v: Any): Option[CypherValue] = v match { + case interval: CalendarInterval => Some(interval.toDuration) + case row: Row => + val pairs: Seq[(String, Any)] = row.schema.fieldNames.map { field => + val index = row.fieldIndex(field) + field -> row.get(index) + } + Some(CypherMap(pairs: _*)) + + case _ => None + } + } + + implicit val sparkCypherValueConverter: CypherValueConverter = SparkCypherValueConverter +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala new file mode 100644 index 0000000000000..28c85cd19bde6 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala @@ -0,0 +1,124 @@ +/* + * 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.spark.cypher.io + +import java.net.URI + +import org.apache.hadoop.fs.FileSystem +import org.apache.spark.cypher.SparkGraphDirectoryStructure +import org.apache.spark.cypher.SparkGraphDirectoryStructure._ +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.conversions.StringEncodingUtilities._ +import org.apache.spark.graph.api.{NodeDataset, RelationshipDataset} +import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.opencypher.okapi.api.graph.{SourceEndNodeKey, SourceIdKey, SourceStartNodeKey} +import org.opencypher.okapi.api.types.{CTNode, CTRelationship} +import org.opencypher.okapi.ir.api.expr.{Property, Var} +import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph + +object ReadWriteGraph { + + case class GraphImporter(sparkSession: SparkSession, path: String, format: String) { + + val directoryStructure: SparkGraphDirectoryStructure = SparkGraphDirectoryStructure(path) + val (labelCombos, relTypes): (Seq[Set[String]], Seq[String]) = { + val fs = FileSystem.get(new URI(path), sparkSession.sparkContext.hadoopConfiguration) + try { + import org.apache.spark.cypher.util.HadoopFSUtils._ + val combos = fs.listDirectories(directoryStructure.pathToNodeDirectory).map(_.toLabelCombo) + val types = fs.listDirectories(directoryStructure.pathToRelationshipDirectory).map(_.toRelationshipType) + combos -> types + } finally { + fs.close() + } + } + + def nodeDatasets: Seq[NodeDataset] = { + labelCombos.map { combo => + val df = sparkSession.read.format(format).load(directoryStructure.pathToNodeTable(combo)) + val propertyMappings = df.columns.collect { + case colName if colName.isPropertyColumnName => colName.toProperty -> colName + }.toMap + NodeDataset( + df, + SourceIdKey.name, + combo, + propertyMappings) + } + } + + def relationshipDatasets: Seq[RelationshipDataset] = { + relTypes.map { relType => + val df = sparkSession.read.format(format).load(directoryStructure.pathToRelationshipTable(relType)) + val propertyMappings = df.columns.collect { + case colName if colName.isPropertyColumnName => colName.toProperty -> colName + }.toMap + RelationshipDataset( + df, + SourceIdKey.name, + SourceStartNodeKey.name, + SourceEndNodeKey.name, + relType, + propertyMappings) + } + } + + } + + implicit class GraphExport(graph: RelationalCypherGraph[DataFrameTable]) { + + def canonicalNodeTable(labels: Set[String]): Dataset[Row] = { + val ct = CTNode(labels) + val v = Var("n")(ct) + val nodeRecords = graph.nodes(v.name, ct, exactLabelMatch = true) + val header = nodeRecords.header + + val idRenaming = header.column(v) -> SourceIdKey.name + val properties: Set[Property] = header.propertiesFor(v) + val propertyRenames = properties.map { p => header.column(p) -> p.key.name.toPropertyColumnName } + + val selectColumns = (idRenaming :: propertyRenames.toList.sortBy { case (_, newName) => newName }).map { + case (oldName, newName) => nodeRecords.table.df.col(oldName).as(newName) + } + + nodeRecords.table.df.select(selectColumns: _*) + } + + def canonicalRelationshipTable(relType: String): Dataset[Row] = { + val ct = CTRelationship(relType) + val v = Var("r")(ct) + val relRecords = graph.relationships(v.name, ct) + val header = relRecords.header + + val idRenaming = header.column(v) -> SourceIdKey.name + val sourceIdRenaming = header.column(header.startNodeFor(v)) -> SourceStartNodeKey.name + val targetIdRenaming = header.column(header.endNodeFor(v)) -> SourceEndNodeKey.name + val properties: Set[Property] = relRecords.header.propertiesFor(v) + val propertyRenames = properties.map { p => relRecords.header.column(p) -> p.key.name.toPropertyColumnName } + + val selectColumns = (idRenaming :: sourceIdRenaming :: targetIdRenaming :: propertyRenames.toList.sortBy { case (_, newName) => newName }).map { + case (oldName, newName) => relRecords.table.df.col(oldName).as(newName) + } + + relRecords.table.df.select(selectColumns: _*) + } + + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphReader.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphReader.scala new file mode 100644 index 0000000000000..b183c78d0be3b --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphReader.scala @@ -0,0 +1,31 @@ +/* + * 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.spark.cypher.io + +import org.apache.spark.cypher.io.ReadWriteGraph.GraphImporter +import org.apache.spark.graph.api.{CypherSession, PropertyGraph, PropertyGraphReader} + +class SparkCypherPropertyGraphReader(session: CypherSession) extends PropertyGraphReader(session) { + + override def load(path: String): PropertyGraph = { + val graphImporter = GraphImporter(session.sparkSession, path, format) + session.createGraph(graphImporter.nodeDatasets.toArray, graphImporter.relationshipDatasets.toArray) + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphWriter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphWriter.scala new file mode 100644 index 0000000000000..643a3d31e3c4d --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphWriter.scala @@ -0,0 +1,48 @@ +/* + * 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.spark.cypher.io + +import org.apache.spark.cypher.SparkCypherSession._ +import org.apache.spark.cypher.SparkGraphDirectoryStructure +import org.apache.spark.cypher.io.ReadWriteGraph._ +import org.apache.spark.graph.api.{PropertyGraph, PropertyGraphWriter} + +class SparkCypherPropertyGraphWriter(graph: PropertyGraph) extends PropertyGraphWriter(graph) { + + override def save(path: String): Unit = { + val relationalGraph = toRelationalGraph(graph) + val graphDirectoryStructure = SparkGraphDirectoryStructure(path) + + relationalGraph.schema.labelCombinations.combos.foreach { combo => + relationalGraph.canonicalNodeTable(combo) + .write + .format(format) + .mode(saveMode) + .save(graphDirectoryStructure.pathToNodeTable(combo)) + } + relationalGraph.schema.relationshipTypes.foreach { relType => + relationalGraph.canonicalRelationshipTable(relType) + .write + .format(format) + .mode(saveMode) + .save(graphDirectoryStructure.pathToRelationshipTable(relType)) + } + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/udfs/TemporalUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/udfs/TemporalUdfs.scala new file mode 100644 index 0000000000000..4ff30273342f3 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/udfs/TemporalUdfs.scala @@ -0,0 +1,146 @@ +/* + * 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.spark.cypher.udfs + +import java.sql.{Date, Timestamp} +import java.time.temporal.{ChronoField, IsoFields, TemporalField} + +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions.udf +import org.apache.spark.unsafe.types.CalendarInterval +import org.opencypher.okapi.impl.exception.UnsupportedOperationException + +import scala.reflect.runtime.universe._ + +object TemporalUdfs { + + /** + * Adds a duration to a date. + * Duration components on a sub-day level are ignored + */ + val dateAdd: UserDefinedFunction = + udf[Date, Date, CalendarInterval]((date: Date, interval: CalendarInterval) => { + if (date == null || interval == null) { + null + } else { + val days = interval.microseconds / CalendarInterval.MICROS_PER_DAY + + val reducedLocalDate = date + .toLocalDate + .plusMonths(interval.months) + .plusDays(days) + + Date.valueOf(reducedLocalDate) + } + }) + + /** + * Subtracts a duration from a date. + * Duration components on a sub-day level are ignored + */ + val dateSubtract: UserDefinedFunction = + udf[Date, Date, CalendarInterval]((date: Date, interval: CalendarInterval) => { + if (date == null || interval == null) { + null + } else { + val days = interval.microseconds / CalendarInterval.MICROS_PER_DAY + + val reducedLocalDate = date + .toLocalDate + .minusMonths(interval.months) + .minusDays(days) + + Date.valueOf(reducedLocalDate) + } + }) + + /** + * Returns the week based year of a given temporal type. + */ + def weekYear[I: TypeTag]: UserDefinedFunction = dateAccessor[I](IsoFields.WEEK_BASED_YEAR) + + /** + * Returns the day of the quarter of a given temporal type. + */ + def dayOfQuarter[I: TypeTag]: UserDefinedFunction = dateAccessor[I](IsoFields.DAY_OF_QUARTER) + + /** + * Returns the day of the week of a given temporal type. + */ + def dayOfWeek[I: TypeTag]: UserDefinedFunction = dateAccessor[I](ChronoField.DAY_OF_WEEK) + + /** + * Returns the milliseconds. + */ + def milliseconds[I: TypeTag]: UserDefinedFunction = timeAccessor[I](ChronoField.MILLI_OF_SECOND) + + /** + * Returns the microseconds. + */ + def microseconds[I: TypeTag]: UserDefinedFunction = timeAccessor[I](ChronoField.MICRO_OF_SECOND) + + def durationAccessor(accessor: String): UserDefinedFunction = udf[java.lang.Long, CalendarInterval]( + (duration: CalendarInterval) => { + if (duration == null) { + null + } else { + val days = duration.microseconds / CalendarInterval.MICROS_PER_DAY + // Note: in cypher days (and weeks) make up their own group, thus we have to exclude them for all values < day + val daysInMicros = days * CalendarInterval.MICROS_PER_DAY + + val l: Long = accessor match { + case "years" => duration.months / 12 + case "quarters" => duration.months / 3 + case "months" => duration.months + case "weeks" => duration.microseconds / CalendarInterval.MICROS_PER_DAY / 7 + case "days" => duration.microseconds / CalendarInterval.MICROS_PER_DAY + case "hours" => (duration.microseconds - daysInMicros ) / CalendarInterval.MICROS_PER_HOUR + case "minutes" => (duration.microseconds - daysInMicros ) / CalendarInterval.MICROS_PER_MINUTE + case "seconds" => (duration.microseconds - daysInMicros ) / CalendarInterval.MICROS_PER_SECOND + case "milliseconds" => (duration.microseconds - daysInMicros ) / CalendarInterval.MICROS_PER_MILLI + case "microseconds" => duration.microseconds - daysInMicros + + case "quartersofyear" => (duration.months / 3) % 4 + case "monthsofquarter" => duration.months % 3 + case "monthsofyear" => duration.months % 12 + case "daysofweek" => (duration.microseconds / CalendarInterval.MICROS_PER_DAY) % 7 + case "minutesofhour" => ((duration.microseconds - daysInMicros )/ CalendarInterval.MICROS_PER_MINUTE) % 60 + case "secondsofminute" => ((duration.microseconds - daysInMicros ) / CalendarInterval.MICROS_PER_SECOND) % 60 + case "millisecondsofsecond" => ((duration.microseconds - daysInMicros ) / CalendarInterval.MICROS_PER_MILLI) % 1000 + case "microsecondsofsecond" => (duration.microseconds - daysInMicros ) % 1000000 + + case other => throw UnsupportedOperationException(s"Unknown Duration accessor: $other") + } + new java.lang.Long(l) + } + } + ) + + private def dateAccessor[I: TypeTag](accessor: TemporalField): UserDefinedFunction = udf[Long, I] { + case d: Date => d.toLocalDate.get(accessor) + case l: Timestamp => l.toLocalDateTime.get(accessor) + case other => throw UnsupportedOperationException(s"Date Accessor '$accessor' is not supported for '$other'.") + } + + private def timeAccessor[I: TypeTag](accessor: TemporalField): UserDefinedFunction = udf[Long, I] { + case l: Timestamp => l.toLocalDateTime.get(accessor) + case other => throw UnsupportedOperationException(s"Time Accessor '$accessor' is not supported for '$other'.") + } +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/cypher/util/HadoopFSUtils.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/util/HadoopFSUtils.scala new file mode 100644 index 0000000000000..a6668b1dd0343 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/util/HadoopFSUtils.scala @@ -0,0 +1,39 @@ +/* + * 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.spark.cypher.util + +import org.apache.hadoop.fs.{FileSystem, Path} + +object HadoopFSUtils { + + implicit class RichHadoopFileSystem(fileSystem: FileSystem) { + + def listDirectories(path: String): Seq[String] = { + val p = new Path(path) + fileSystem + .listStatus(p) + .collect { case item if item.isDirectory => + item.getPath.getName + } + .toSeq + } + + } + +} diff --git a/graph/cypher/src/test/java/org/apache/spark/graph/api/SparkCypherPropertyGraphSuite.java b/graph/cypher/src/test/java/org/apache/spark/graph/api/SparkCypherPropertyGraphSuite.java new file mode 100644 index 0000000000000..4ac41d57d36f9 --- /dev/null +++ b/graph/cypher/src/test/java/org/apache/spark/graph/api/SparkCypherPropertyGraphSuite.java @@ -0,0 +1,28 @@ +/* + * 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.spark.graph.api; + +import org.apache.spark.cypher.SparkCypherSession; +import org.apache.spark.sql.SparkSession; + +public class SparkCypherPropertyGraphSuite extends JavaPropertyGraphSuite { + @Override + CypherSession getCypherSession(SparkSession sparkSession) { + return new SparkCypherSession(sparkSession); + } +} diff --git a/graph/cypher/src/test/resources/tck/failing_blacklist b/graph/cypher/src/test/resources/tck/failing_blacklist new file mode 100644 index 0000000000000..7661161bd429b --- /dev/null +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -0,0 +1,233 @@ +Feature "WithAcceptance": Scenario "A simple pattern with one bound endpoint" +Feature "VarLengthAcceptance2": Scenario "Handling relationships that are already bound in variable length paths" +Feature "VarLengthAcceptance": Scenario "Handling unbounded variable length match" +Feature "VarLengthAcceptance": Scenario "Handling explicitly unbounded variable length match" +Feature "VarLengthAcceptance": Scenario "Handling lower bounded variable length match 1" +Feature "VarLengthAcceptance": Scenario "Handling lower bounded variable length match 2" +Feature "VarLengthAcceptance": Scenario "Handling lower bounded variable length match 3" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, zero length 1" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, zero length 2" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, single length 1" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, single length 2" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, longer 1" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, longer 2" +Feature "VarLengthAcceptance": Scenario "Handling a variable length relationship and a standard relationship in chain, longer 3" +Feature "VarLengthAcceptance": Scenario "Handling mixed relationship patterns and directions 1" +Feature "VarLengthAcceptance": Scenario "Handling mixed relationship patterns and directions 2" +Feature "VarLengthAcceptance": Scenario "Handling mixed relationship patterns 1" +Feature "VarLengthAcceptance": Scenario "Handling mixed relationship patterns 2" +Feature "UnwindAcceptance": Scenario "Unwinding a collected expression" +Feature "UnwindAcceptance": Scenario "Unwind does not remove variables from scope" +Feature "TypeConversionFunctions": Scenario "`toInteger()` handling mixed number types" +Feature "TypeConversionFunctions": Scenario "`toInteger()` handling Any type" +Feature "TypeConversionFunctions": Scenario "`toInteger()` on a list of strings" +Feature "TypeConversionFunctions": Scenario "`toFloat()` on mixed number types" +Feature "TypeConversionFunctions": Scenario "`toFloat()` handling Any type" +Feature "TypeConversionFunctions": Scenario "`toFloat()` on a list of strings" +Feature "TypeConversionFunctions": Scenario "`toString()` should work on Any type" +Feature "TypeConversionFunctions": Scenario "`toString()` on a list of integers" +Feature "TypeConversionFunctions": Scenario "`toBoolean()` on invalid types #1" +Feature "TypeConversionFunctions": Scenario "`toBoolean()` on invalid types #2" +Feature "TypeConversionFunctions": Scenario "`toBoolean()` on invalid types #3" +Feature "TypeConversionFunctions": Scenario "`toBoolean()` on invalid types #4" +Feature "StartsWithAcceptance": Scenario "Handling non-string operands for STARTS WITH" +Feature "StartsWithAcceptance": Scenario "Handling non-string operands for CONTAINS" +Feature "StartsWithAcceptance": Scenario "Handling non-string operands for ENDS WITH" +Feature "SkipLimitAcceptanceTest": Scenario "SKIP with an expression that does not depend on variables" +Feature "SkipLimitAcceptanceTest": Scenario "LIMIT with an expression that does not depend on variables" +Feature "ReturnAcceptance2": Scenario "Return all variables" +Feature "ReturnAcceptance2": Scenario "Setting and returning the size of a list property" +Feature "ReturnAcceptance2": Scenario "Concatenating and returning the size of literal lists" +Feature "ReturnAcceptance2": Scenario "Returning nested expressions based on list property" +Feature "ReturnAcceptance2": Scenario "Arithmetic expressions should propagate null values" +Feature "ReturnAcceptance2": Scenario "Projecting an arithmetic expression with aggregation" +Feature "ReturnAcceptance2": Scenario "Multiple aliasing and backreferencing" +Feature "ReturnAcceptance2": Scenario "Reusing variable names" +Feature "ReturnAcceptance2": Scenario "Concatenating lists of same type" +Feature "ReturnAcceptance2": Scenario "Appending lists of same type" +Feature "ReturnAcceptance2": Scenario "DISTINCT inside aggregation should work with nested lists of maps in maps" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure that takes no arguments" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure that takes no arguments" +Feature "ProcedureCallAcceptance": Scenario "Calling the same procedure twice using the same outputs in each call" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to VOID procedure that takes no arguments" +Feature "ProcedureCallAcceptance": Scenario "In-query call to VOID procedure that takes no arguments" +Feature "ProcedureCallAcceptance": Scenario "In-query call to VOID procedure does not consume rows" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to VOID procedure that takes no arguments, called with implicit arguments" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure that takes no arguments and yields no results" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure that takes no arguments and yields no results" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure that takes no arguments and yields no results, called with implicit arguments" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with explicit arguments" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with explicit arguments that drops all result fields" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with explicit arguments" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with implicit arguments" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with argument of type NUMBER accepts value of type INTEGER" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with argument of type NUMBER accepts value of type INTEGER" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with argument of type NUMBER accepts value of type FLOAT" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with argument of type NUMBER accepts value of type FLOAT" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with argument of type FLOAT accepts value of type INTEGER" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with argument of type FLOAT accepts value of type INTEGER" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with argument of type INTEGER accepts value of type FLOAT" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with argument of type INTEGER accepts value of type FLOAT" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure with null argument" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure with null argument" +Feature "PatternComprehension": Scenario "Pattern comprehension and ORDER BY" +Feature "PatternComprehension": Scenario "Returning a pattern comprehension" +Feature "PatternComprehension": Scenario "Returning a pattern comprehension with label predicate" +Feature "PatternComprehension": Scenario "Returning a pattern comprehension with bound nodes" +Feature "PatternComprehension": Scenario "Using a pattern comprehension in a WITH" +Feature "PatternComprehension": Scenario "Using a variable-length pattern comprehension in a WITH" +Feature "PatternComprehension": Scenario "Using pattern comprehension in RETURN" +Feature "PatternComprehension": Scenario "Aggregating on pattern comprehension" +Feature "PatternComprehension": Scenario "Using pattern comprehension to test existence" +Feature "PatternComprehension": Scenario "Pattern comprehension inside list comprehension" +Feature "PatternComprehension": Scenario "Get node degree via size of pattern comprehension" +Feature "PatternComprehension": Scenario "Get node degree via size of pattern comprehension that specifies a relationship type" +Feature "PatternComprehension": Scenario "Get node degree via size of pattern comprehension that specifies multiple relationship types" +Feature "PatternComprehension": Scenario "Introducing new node variable in pattern comprehension" +Feature "PatternComprehension": Scenario "Introducing new relationship variable in pattern comprehension" +Feature "PathEquality": Scenario "Direction of traversed relationship is not significant for path equality, simple" +Feature "OrderByAcceptance": Scenario "ORDER BY with negative parameter for LIMIT should not generate errors" +Feature "OptionalMatchAcceptance": Scenario "Named paths in optional matches" +Feature "OptionalMatchAcceptance": Scenario "Named paths inside optional matches with node predicates" +Feature "OptionalMatchAcceptance": Scenario "Variable length optional relationships" +Feature "OptionalMatchAcceptance": Scenario "Variable length optional relationships with length predicates" +Feature "OptionalMatchAcceptance": Scenario "Variable length optional relationships with bound nodes" +Feature "OptionalMatchAcceptance": Scenario "Variable length optional relationships with bound nodes, no matches" +Feature "MatchAcceptance2": Scenario "Aggregation with named paths" +Feature "MatchAcceptance2": Scenario "Simple variable length pattern" +Feature "MatchAcceptance2": Scenario "Variable length relationship without lower bound" +Feature "MatchAcceptance2": Scenario "Variable length relationship without bounds" +Feature "MatchAcceptance2": Scenario "Optionally matching named paths" +Feature "MatchAcceptance2": Scenario "Optionally matching named paths with single and variable length patterns" +Feature "MatchAcceptance2": Scenario "Optionally matching named paths with variable length patterns" +Feature "MatchAcceptance2": Scenario "Matching variable length patterns from a bound node" +Feature "MatchAcceptance2": Scenario "Variable length relationship in OPTIONAL MATCH" +Feature "MatchAcceptance2": Scenario "Handling direction of named paths" +Feature "MatchAcceptance2": Scenario "Handling fixed-length variable length pattern" +Feature "MatchAcceptance2": Scenario "Zero-length named path" +Feature "MatchAcceptance2": Scenario "Variable-length named path" +Feature "MatchAcceptance2": Scenario "Matching using a relationship that is already bound" +Feature "MatchAcceptance2": Scenario "Matching using a relationship that is already bound, in conjunction with aggregation" +Feature "MatchAcceptance2": Scenario "Matching using a relationship that is already bound, in conjunction with aggregation and ORDER BY" +Feature "MatchAcceptance2": Scenario "Matching with LIMIT and optionally matching using a relationship that is already bound" +Feature "MatchAcceptance2": Scenario "Matching with LIMIT and optionally matching using a relationship and node that are both already bound" +Feature "MatchAcceptance2": Scenario "Matching with LIMIT, then matching again using a relationship and node that are both already bound along with an additional predicate" +Feature "MatchAcceptance2": Scenario "Matching with LIMIT and predicates, then matching again using a relationship and node that are both already bound along with a duplicate predicate" +Feature "MatchAcceptance2": Scenario "Matching twice with conflicting relationship types on same relationship" +Feature "MatchAcceptance2": Scenario "Matching twice with duplicate relationship types on same relationship" +Feature "MatchAcceptance2": Scenario "Matching relationships into a list and matching variable length using the list" +Feature "MatchAcceptance2": Scenario "Matching relationships into a list and matching variable length using the list, with bound nodes" +Feature "MatchAcceptance2": Scenario "Matching relationships into a list and matching variable length using the list, with bound nodes, wrong direction" +Feature "MatchAcceptance2": Scenario "Matching and optionally matching with bound nodes in reverse direction" +Feature "MatchAcceptance2": Scenario "Matching and optionally matching with unbound nodes and equality predicate in reverse direction" +Feature "MatchAcceptance2": Scenario "Matching variable length pattern with property predicate" +Feature "MatchAcceptance2": Scenario "Variable length pattern checking labels on endnodes" +Feature "MatchAcceptance2": Scenario "Variable length pattern with label predicate on both sides" +Feature "MatchAcceptance2": Scenario "Undirected named path" +Feature "MatchAcceptance2": Scenario "Named path with WITH" +Feature "MatchAcceptance2": Scenario "Named path with alternating directed/undirected relationships" +Feature "MatchAcceptance2": Scenario "Named path with multiple alternating directed/undirected relationships" +Feature "MatchAcceptance2": Scenario "Named path with undirected fixed variable length pattern" +Feature "MatchAcceptance2": Scenario "Variable length relationship variables are lists of relationships" +Feature "MatchAcceptance2": Scenario "Variable length patterns and nulls" +Feature "MatchAcceptance2": Scenario "Projecting a list of nodes and relationships" +Feature "MatchAcceptance2": Scenario "Projecting a map of nodes and relationships" +Feature "MatchAcceptance2": Scenario "Respecting direction when matching existing path" +Feature "MatchAcceptance2": Scenario "Respecting direction when matching non-existent path" +Feature "MatchAcceptance2": Scenario "Respecting direction when matching non-existent path with multiple directions" +Feature "MatchAcceptance2": Scenario "Matching path with both directions should respect other directions" +Feature "MatchAcceptance2": Scenario "Matching path with multiple bidirectional relationships" +Feature "MatchAcceptance2": Scenario "Matching longer variable length paths" +Feature "MatchAcceptance2": Scenario "Counting rows after MATCH, MERGE, OPTIONAL MATCH" +Feature "MatchAcceptance": Scenario "Path query should return results in written order" +Feature "MatchAcceptance": Scenario "Longer path query should return results in written order" +Feature "MatchAcceptance": Scenario "Return a simple path" +Feature "MatchAcceptance": Scenario "Return a three node path" +Feature "MatchAcceptance": Scenario "Do not return anything because path length does not match" +Feature "MatchAcceptance": Scenario "Pass the path length test" +Feature "MatchAcceptance": Scenario "Return relationships by fetching them from the path - starting from the end" +Feature "MatchAcceptance": Scenario "Return relationships by fetching them from the path" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - wrong way" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list" +Feature "MatchAcceptance": Scenario "Return a var length path" +Feature "MatchAcceptance": Scenario "Return a var length path of length zero" +Feature "MatchAcceptance": Scenario "Return a named var length path of length zero" +Feature "ListComprehension": Scenario "Returning a list comprehension" +Feature "ListComprehension": Scenario "Using a list comprehension in a WITH" +Feature "ListComprehension": Scenario "Using a list comprehension in a WHERE" +Feature "FunctionsAcceptance": Scenario "Functions should return null if they get path containing unbound" +Feature "FunctionsAcceptance": Scenario "`split()`" +Feature "FunctionsAcceptance": Scenario "`reverse()`" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` #1" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` #2" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` #3" +Feature "FunctionsAcceptance": Scenario "`percentileCont()` #1" +Feature "FunctionsAcceptance": Scenario "`percentileCont()` #2" +Feature "FunctionsAcceptance": Scenario "`percentileCont()` #3" +Feature "FunctionsAcceptance": Scenario "`type()` handling Any type" +Feature "FunctionsAcceptance": Scenario "`labels()` should accept type Any" +Feature "ExpressionAcceptance": Scenario "Execute n['name'] in read queries" +Feature "ExpressionAcceptance": Scenario "Execute n['name'] in update queries" +Feature "ExpressionAcceptance": Scenario "Use dynamic property lookup based on parameters when there is no type information" +Feature "ExpressionAcceptance": Scenario "Use dynamic property lookup based on parameters when there is lhs type information" +Feature "ExpressionAcceptance": Scenario "Use dynamic property lookup based on parameters when there is rhs type information" +Feature "EqualsAcceptance": Scenario "Any-typed string comparison" +Feature "Comparability": Scenario "Comparing strings and integers using > in an AND'd predicate" +Feature "Comparability": Scenario "Comparing strings and integers using > in a OR'd predicate" +Feature "Comparability": Scenario "Comparing across types yields null, except numbers #1" +Feature "Comparability": Scenario "Comparing across types yields null, except numbers #2" +Feature "Comparability": Scenario "Comparing across types yields null, except numbers #3" +Feature "Comparability": Scenario "Comparing across types yields null, except numbers #4" +Feature "ColumnNameAcceptance": Scenario "Keeping used expression 2" +Feature "ColumnNameAcceptance": Scenario "Keeping used expression 3" +Feature "AggregationAcceptance": Scenario "Handle aggregation on functions" +Feature "AggregationAcceptance": Scenario "Aggregates in aggregates" +Feature "AggregationAcceptance": Scenario "Multiple aggregates on same variable" +Feature "AggregationAcceptance": Scenario "Aggregation of named paths" +Feature "AggregationAcceptance": Scenario "Aggregation with `min()`" +Feature "AggregationAcceptance": Scenario "Handle subexpression in aggregation also occurring as standalone expression with nested aggregation in a literal map" +Feature "AggregationAcceptance": Scenario "Projection during aggregation in WITH before MERGE and after WITH with predicate" +Feature "Aggregation": Scenario "`min()` over mixed values" +Feature "Aggregation": Scenario "`max()` over mixed values" +Feature "Aggregation": Scenario "`max()` over mixed numeric values" +Feature "ListOperations": Scenario "IN should return true if correct list found despite other lists having nulls" +Feature "ListOperations": Scenario "Size of list comprehension" +Feature "ListOperations": Scenario "IN should return false when matching a number with a string - list version" +Feature "ListOperations": Scenario "IN should return false when matching a number with a string" +Feature "ListOperations": Scenario "IN should return true when LHS and RHS contain a nested list - singleton version" +Feature "ListOperations": Scenario "Equality between different nested lists with null should return false" +Feature "ListOperations": Scenario "IN should return null if comparison with null is required, list version 2" +Feature "ListOperations": Scenario "IN should work with an empty list when comparing nested lists" +Feature "ListOperations": Scenario "IN should return null if LHS and RHS are null - list version" +Feature "ListOperations": Scenario "Equality between list and literal should return false" +Feature "ListOperations": Scenario "IN should work with an empty list in the presence of other list elements: matching" +Feature "ListOperations": Scenario "Setting and returning the size of a list property" +Feature "ListOperations": Scenario "Collect and extract using a list comprehension" +Feature "ListOperations": Scenario "IN with different length lists should return false" +Feature "ListOperations": Scenario "IN should return true when both LHS and RHS contain nested lists" +Feature "ListOperations": Scenario "IN should return true when LHS and RHS contain nested list with multiple empty lists" +Feature "ListOperations": Scenario "IN should return true when types of LHS and RHS match - list" +Feature "ListOperations": Scenario "IN should return true when LHS and RHS contain a nested list" +Feature "ListOperations": Scenario "IN should return true when types of LHS and RHS match - singleton list" +Feature "ListOperations": Scenario "IN should return null when comparing two so-called identical lists where one element is null" +Feature "ListOperations": Scenario "IN should return null if comparison with null is required, list version" +Feature "ListOperations": Scenario "IN should return false if different length lists with nested elements compared, even if the extra element is null" +Feature "ListOperations": Scenario "Equality between different lists with null should return false" +Feature "ListOperations": Scenario "Returning nested expressions based on list property" +Feature "ListOperations": Scenario "Equality between almost equal nested lists with null should return null" +Feature "ListOperations": Scenario "Equality of nested lists of different length should return false despite nulls" +Feature "ListOperations": Scenario "IN should return false if different length lists compared, even if the extra element is null" +Feature "ListOperations": Scenario "Equality between almost equal lists with null should return null" +Feature "ListOperations": Scenario "IN should return true when both LHS and RHS contain a nested list alongside a scalar element" +Feature "ListOperations": Scenario "IN should return true if correct list found despite null being another element within containing list" +Feature "ListOperations": Scenario "IN should return false when order of elements in LHS list and RHS list don't match" +Feature "ListOperations": Scenario "Collect and filter using a list comprehension" +Feature "ListOperations": Scenario "List slice with negative range" +Feature "ListOperations": Scenario "List slice with parameterised invalid range" +Feature "ListOperations": Scenario "List slice with invalid range" +Feature "ListOperations": Scenario "List slice with exceeding range" +Feature "NullAcceptance": Scenario "Property existence check on null node" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - directed, one way" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected, starting from one extreme" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected, starting from two extremes" diff --git a/graph/cypher/src/test/resources/tck/failure_reporting_blacklist b/graph/cypher/src/test/resources/tck/failure_reporting_blacklist new file mode 100644 index 0000000000000..51778d2a176e9 --- /dev/null +++ b/graph/cypher/src/test/resources/tck/failure_reporting_blacklist @@ -0,0 +1,118 @@ +Feature "MiscellaneousErrorAcceptance": Scenario "Failing on incorrect unicode literal" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing on aggregation in WHERE" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing on aggregation in ORDER BY after RETURN" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing on aggregation in ORDER BY after WITH" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when not aliasing expressions in WITH" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using undefined variable in pattern" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using a path variable that is already bound" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using a list as a node" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using a variable length relationship as a single relationship" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when UNION has different columns" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when mixing UNION and UNION ALL" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when setting a list of maps as a property" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when multiple columns have the same name" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using RETURN * without variables in scope" +Feature "SemanticErrorAcceptance": Scenario "Failing when returning an undefined variable" +Feature "SemanticErrorAcceptance": Scenario "Failing when comparing to an undefined variable" +Feature "SemanticErrorAcceptance": Scenario "Failing when using IN on a string literal" +Feature "SemanticErrorAcceptance": Scenario "Failing when using IN on an integer literal" +Feature "SemanticErrorAcceptance": Scenario "Failing when using IN on a float literal" +Feature "SemanticErrorAcceptance": Scenario "Failing when using IN on a boolean literal" +Feature "SemanticErrorAcceptance": Scenario "Failing when a node is used as a relationship" +Feature "SemanticErrorAcceptance": Scenario "Failing when a relationship is used as a node" +Feature "SemanticErrorAcceptance": Scenario "Failing when using `type()` on a node" +Feature "SemanticErrorAcceptance": Scenario "Failing when using `length()` on a node" +Feature "SemanticErrorAcceptance": Scenario "Failing when re-using a relationship in the same pattern" +Feature "SemanticErrorAcceptance": Scenario "Failing when using NOT on string literal" +Feature "SemanticErrorAcceptance": Scenario "Failing when using parameter as node predicate in MATCH" +Feature "SemanticErrorAcceptance": Scenario "Failing when using parameter as relationship predicate in MATCH" +Feature "SemanticErrorAcceptance": Scenario "Failing when using undefined variable in ON MATCH" +Feature "SemanticErrorAcceptance": Scenario "Failing when using MATCH after OPTIONAL MATCH" +Feature "SemanticErrorAcceptance": Scenario "Failing when float value is too large" +Feature "SemanticErrorAcceptance": Scenario "Handling property access on the Any type" +Feature "SemanticErrorAcceptance": Scenario "Failing when performing property access on a non-map 1" +Feature "SemanticErrorAcceptance": Scenario "Failing when performing property access on a non-map 2" +Feature "SemanticErrorAcceptance": Scenario "Failing when checking existence of a non-property and non-pattern" +Feature "SemanticErrorAcceptance": Scenario "Bad arguments for `range()`" +Feature "SemanticErrorAcceptance": Scenario "Fail for invalid Unicode hyphen in subtraction" +Feature "SemanticErrorAcceptance": Scenario "Failing for `size()` on paths" +Feature "SemanticErrorAcceptance": Scenario "Failing when using aggregation in list comprehension" +Feature "SemanticErrorAcceptance": Scenario "Failing when using non-constants in SKIP" +Feature "SemanticErrorAcceptance": Scenario "Failing when using negative value in SKIP" +Feature "SemanticErrorAcceptance": Scenario "Failing when using non-constants in LIMIT" +Feature "SemanticErrorAcceptance": Scenario "Failing when using negative value in LIMIT" +Feature "SemanticErrorAcceptance": Scenario "Failing when using floating point in LIMIT" +Feature "SemanticErrorAcceptance": Scenario "Failing when creating relationship without type" +Feature "SemanticErrorAcceptance": Scenario "Failing when merging relationship without type" +Feature "SemanticErrorAcceptance": Scenario "Failing when merging relationship without type, no colon" +Feature "SemanticErrorAcceptance": Scenario "Failing when creating relationship with more than one type" +Feature "SemanticErrorAcceptance": Scenario "Failing when merging relationship with more than one type" +Feature "SyntaxErrorAcceptance": Scenario "Using a non-existent function" +Feature "SyntaxErrorAcceptance": Scenario "Using `rand()` in aggregations" +Feature "SyntaxErrorAcceptance": Scenario "Supplying invalid hexadecimal literal 1" +Feature "SyntaxErrorAcceptance": Scenario "Supplying invalid hexadecimal literal 2" +Feature "VarLengthAcceptance": Scenario "Fail when asterisk operator is missing" +Feature "VarLengthAcceptance": Scenario "Fail on negative bound" +Feature "MatchAcceptance2": Scenario "Fail when using property access on primitive type" +Feature "ExpressionAcceptance": Scenario "Fail at runtime when attempting to index with an Int into a Map" +Feature "ExpressionAcceptance": Scenario "Fail at runtime when trying to index into a map with a non-string" +Feature "ExpressionAcceptance": Scenario "Fail at runtime when attempting to index with a String into a Collection" +Feature "ExpressionAcceptance": Scenario "Fail at runtime when trying to index into a list with a list" +Feature "ExpressionAcceptance": Scenario "Fail at compile time when attempting to index with a non-integer into a list" +Feature "ExpressionAcceptance": Scenario "Fail at runtime when trying to index something which is not a map or collection" +Feature "FunctionsAcceptance": Scenario "`properties()` failing on an integer literal" +Feature "FunctionsAcceptance": Scenario "`properties()` failing on a string literal" +Feature "FunctionsAcceptance": Scenario "`properties()` failing on a list of booleans" +Feature "FunctionsAcceptance": Scenario "`percentileCont()` failing on bad arguments #1" +Feature "FunctionsAcceptance": Scenario "`percentileCont()` failing on bad arguments #2" +Feature "FunctionsAcceptance": Scenario "`percentileCont()` failing on bad arguments #3" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` failing on bad arguments #1" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` failing on bad arguments #2" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` failing on bad arguments #3" +Feature "FunctionsAcceptance": Scenario "`percentileDisc()` failing in more involved query" +Feature "FunctionsAcceptance": Scenario "`labels()` failing on a path" +Feature "FunctionsAcceptance": Scenario "`labels()` failing on invalid arguments" +Feature "TypeConversionFunctions": Scenario "`toString()` failing on invalid arguments #1" +Feature "TypeConversionFunctions": Scenario "`toString()` failing on invalid arguments #2" +Feature "TypeConversionFunctions": Scenario "`toString()` failing on invalid arguments #3" +Feature "TypeConversionFunctions": Scenario "`toString()` failing on invalid arguments #4" +Feature "TypeConversionFunctions": Scenario "`toString()` failing on invalid arguments #5" +Feature "TypeConversionFunctions": Scenario "`toInteger()` failing on invalid arguments #1" +Feature "TypeConversionFunctions": Scenario "`toInteger()` failing on invalid arguments #2" +Feature "TypeConversionFunctions": Scenario "`toInteger()` failing on invalid arguments #3" +Feature "TypeConversionFunctions": Scenario "`toInteger()` failing on invalid arguments #4" +Feature "TypeConversionFunctions": Scenario "`toInteger()` failing on invalid arguments #5" +Feature "TypeConversionFunctions": Scenario "`toInteger()` failing on invalid arguments #6" +Feature "TypeConversionFunctions": Scenario "`toFloat()` failing on invalid arguments #1" +Feature "TypeConversionFunctions": Scenario "`toFloat()` failing on invalid arguments #2" +Feature "TypeConversionFunctions": Scenario "`toFloat()` failing on invalid arguments #3" +Feature "TypeConversionFunctions": Scenario "`toFloat()` failing on invalid arguments #4" +Feature "TypeConversionFunctions": Scenario "`toFloat()` failing on invalid arguments #5" +Feature "TypeConversionFunctions": Scenario "`toFloat()` failing on invalid arguments #6" +Feature "FunctionsAcceptance": Scenario "`type()` failing on invalid arguments #1" +Feature "FunctionsAcceptance": Scenario "`type()` failing on invalid arguments #2" +Feature "FunctionsAcceptance": Scenario "`type()` failing on invalid arguments #3" +Feature "FunctionsAcceptance": Scenario "`type()` failing on invalid arguments #4" +Feature "FunctionsAcceptance": Scenario "`type()` failing on invalid arguments #5" +Feature "SkipLimitAcceptanceTest": Scenario "SKIP with an expression that depends on variables should fail" +Feature "SkipLimitAcceptanceTest": Scenario "LIMIT with an expression that depends on variables should fail" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure that takes arguments fails when trying to pass them implicitly" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure should fail if input type is wrong" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure should fail if input type is wrong" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure should fail if explicit argument is missing" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure should fail if explicit argument is missing" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure should fail if too many explicit argument are given" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure should fail if too many explicit argument are given" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to procedure should fail if implicit argument is missing" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure that has outputs fails if no outputs are yielded" +Feature "ProcedureCallAcceptance": Scenario "In-query call to procedure that both takes arguments and has outputs fails if the arguments are passed implicitly and no outputs are yielded" +Feature "ProcedureCallAcceptance": Scenario "Standalone call to unknown procedure should fail" +Feature "ProcedureCallAcceptance": Scenario "In-query call to unknown procedure should fail" +Feature "ProcedureCallAcceptance": Scenario "In-query procedure call should fail if shadowing an already bound variable" +Feature "ProcedureCallAcceptance": Scenario "In-query procedure call should fail if one of the argument expressions uses an aggregation function" +Feature "OrderByAcceptance": Scenario "ORDER BY with a negative LIMIT should fail with a syntax exception" +Feature "MatchAcceptance2": Scenario "Do not fail when evaluating predicates with illegal operations if the OR'd predicate evaluates to true" +Feature "ListOperations": Scenario "Fail at runtime when attempting to index with a String into a List" +Feature "ListOperations": Scenario "Fail at runtime when trying to index into a list with a list" +Feature "ListOperations": Scenario "Fail at compile time when attempting to index with a non-integer into a list" +Feature "ExpressionAcceptance": Scenario "Fail at runtime when trying to index something which is not a map or list" diff --git a/graph/cypher/src/test/resources/tck/temporal_blacklist b/graph/cypher/src/test/resources/tck/temporal_blacklist new file mode 100644 index 0000000000000..cf491a6c39337 --- /dev/null +++ b/graph/cypher/src/test/resources/tck/temporal_blacklist @@ -0,0 +1,940 @@ +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from time #1" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from time #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from time #3" +Feature "TemporalAccessorAcceptance": Scenario "Should provide accessors for time" +Feature "TemporalArithmeticAcceptance": Scenario "Should multiply or divide durations by numbers #1" +Feature "TemporalArithmeticAcceptance": Scenario "Should multiply or divide durations by numbers #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should multiply or divide durations by numbers #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #8" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #9" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #11" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #12" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #14" +Feature "TemporalCreateAcceptance": Scenario "Should construct week datetime #15" +Feature "TemporalSelectAcceptance": Scenario "Should select date into local date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select date into local date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select date into local date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select date into local date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select date into local date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select date into local date time #6" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #1" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #2" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #3" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #4" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #5" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #6" +Feature "TemporalParseAcceptance": Scenario "Should parse local time from string #7" +Feature "TemporalToStringAcceptance": Scenario "Should serialize local date time" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #8" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #9" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #10" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #11" +Feature "TemporalSelectAcceptance": Scenario "Should select date into date time #12" +Feature "TemporalCreateAcceptance": Scenario "Should store duration #1" +Feature "TemporalCreateAcceptance": Scenario "Should store duration #2" +Feature "TemporalCreateAcceptance": Scenario "Should store duration #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time from epoch" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #8" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #9" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #11" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #12" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #14" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #15" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #16" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #17" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #18" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #19" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #20" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #21" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #22" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #23" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #24" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #25" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #26" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #27" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #28" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #29" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with default time zone #30" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from local time #1" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from local time #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from local time #3" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #3" +Feature "TemporalParseAcceptance": Scenario "Should parse date time with named time zone from string #1" +Feature "TemporalParseAcceptance": Scenario "Should parse date time with named time zone from string #2" +Feature "TemporalParseAcceptance": Scenario "Should parse date time with named time zone from string #3" +Feature "TemporalParseAcceptance": Scenario "Should parse date time with named time zone from string #4" +Feature "TemporalParseAcceptance": Scenario "Should parse date time with named time zone from string #5" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #1" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #2" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #3" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #4" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #5" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #6" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #7" +Feature "TemporalParseAcceptance": Scenario "Should parse time from string #8" +Feature "DurationBetweenAcceptance": Scenario "Should handle large durations in seconds" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #6" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #7" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #8" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #9" +Feature "TemporalSelectAcceptance": Scenario "Should datetime into date time #10" +Feature "TemporalToStringAcceptance": Scenario "Should serialize local time" +Feature "DurationBetweenAcceptance": Scenario "Should handle durations at daylight saving time day #1" +Feature "DurationBetweenAcceptance": Scenario "Should handle durations at daylight saving time day #2" +Feature "DurationBetweenAcceptance": Scenario "Should handle durations at daylight saving time day #3" +Feature "DurationBetweenAcceptance": Scenario "Should handle durations at daylight saving time day #4" +Feature "DurationBetweenAcceptance": Scenario "Should handle durations at daylight saving time day #5" +Feature "DurationBetweenAcceptance": Scenario "Should handle durations at daylight saving time day #6" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #2" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #3" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #4" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #5" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #6" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #7" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #8" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #9" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #10" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #11" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #12" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #14" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #15" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #16" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #18" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #17" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #19" +Feature "TemporalCreateAcceptance": Scenario "Should propagate null #20" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #1" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #2" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #3" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #4" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #5" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #6" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #7" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #8" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #9" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #10" +Feature "TemporalToStringAcceptance": Scenario "Should serialize duration #11" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #8" +Feature "TemporalCreateAcceptance": Scenario "Should construct duration #9" +Feature "DurationBetweenAcceptance": Scenario "Should propagate null #1" +Feature "DurationBetweenAcceptance": Scenario "Should propagate null #2" +Feature "DurationBetweenAcceptance": Scenario "Should propagate null #3" +Feature "DurationBetweenAcceptance": Scenario "Should propagate null #4" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #8" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #9" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #10" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #11" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #12" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #13" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #14" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #15" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #16" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #17" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #18" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #19" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #20" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #21" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #22" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #23" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #24" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #25" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #26" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #27" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #28" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #29" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #30" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #31" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #32" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #33" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #34" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #35" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #36" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #37" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #38" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #39" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #40" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #41" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #42" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #43" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #44" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #45" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #46" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #47" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into date time #48" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #1" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #2" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #3" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #4" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #5" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #6" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #7" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #8" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #9" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #10" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #11" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #12" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #13" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #14" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #15" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #16" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #17" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #18" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #19" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #20" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #21" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #22" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #23" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #24" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #25" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #26" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #27" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #28" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #29" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #30" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #31" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #32" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #33" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #34" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #35" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #36" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #37" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #38" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #39" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #40" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #41" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #42" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #43" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #44" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #45" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #46" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #47" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate time #48" +Feature "TemporalCreateAcceptance": Scenario "Should construct week date #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct week date #14" +Feature "TemporalCreateAcceptance": Scenario "Should construct week date #15" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #8" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #9" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #10" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #11" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #12" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #13" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #14" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #15" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #16" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #17" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #18" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #19" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #20" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #21" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #22" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #23" +Feature "TemporalSelectAcceptance": Scenario "Should select date and time into local date time #24" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #1" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #3" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #4" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #5" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #6" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #7" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #8" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract durations #9" +Feature "TemporalAccessorAcceptance": Scenario "Should provide accessors for local date time" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct local time #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct temporal with time offset with second precision #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct temporal with time offset with second precision #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct temporal with time offset with second precision #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct temporal with time offset with second precision #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #1" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #2" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #3" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #5" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #6" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #7" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #8" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #9" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #10" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #11" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #12" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #13" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #14" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #15" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #16" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #17" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #18" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #19" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #20" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #21" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #22" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #23" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #24" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in seconds #25" +Feature "TemporalToStringAcceptance": Scenario "Should serialize date" +Feature "TemporalSelectAcceptance": Scenario "Should select time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select time #8" +Feature "TemporalSelectAcceptance": Scenario "Should select time #9" +Feature "TemporalSelectAcceptance": Scenario "Should select time #10" +Feature "TemporalSelectAcceptance": Scenario "Should select time #11" +Feature "TemporalSelectAcceptance": Scenario "Should select time #12" +Feature "TemporalSelectAcceptance": Scenario "Should select time #13" +Feature "TemporalSelectAcceptance": Scenario "Should select time #14" +Feature "TemporalSelectAcceptance": Scenario "Should select time #15" +Feature "TemporalSelectAcceptance": Scenario "Should select time #16" +Feature "TemporalSelectAcceptance": Scenario "Should select time #17" +Feature "TemporalSelectAcceptance": Scenario "Should select time #18" +Feature "TemporalSelectAcceptance": Scenario "Should select time #19" +Feature "TemporalSelectAcceptance": Scenario "Should select time #20" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #1" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #2" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #3" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #4" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #5" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #6" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #7" +Feature "TemporalComparisonAcceptance": Scenario "Should compare durations for equality #8" +Feature "TemporalCreateAcceptance": Scenario "Should store time #1" +Feature "TemporalCreateAcceptance": Scenario "Should store time #2" +Feature "TemporalCreateAcceptance": Scenario "Should store time #3" +Feature "TemporalCreateAcceptance": Scenario "Should store local date time #1" +Feature "TemporalCreateAcceptance": Scenario "Should store local date time #2" +Feature "TemporalCreateAcceptance": Scenario "Should store local date time #3" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #2" +Feature "TemporalCreateAcceptance": Scenario "Should store local time #1" +Feature "TemporalCreateAcceptance": Scenario "Should store local time #2" +Feature "TemporalCreateAcceptance": Scenario "Should store local time #3" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date time #1" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date time #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date time #3" +Feature "TemporalComparisonAcceptance": Scenario "Should compare date times #1" +Feature "TemporalComparisonAcceptance": Scenario "Should compare date times #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #8" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #9" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #11" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #12" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct time #14" +Feature "DurationBetweenAcceptance": Scenario "Should handle large durations" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #1" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #2" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #3" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #4" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #5" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #6" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #7" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #8" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #9" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #10" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #11" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #12" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #13" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #14" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #15" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #16" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #17" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #18" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #19" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #20" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #21" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #22" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #23" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #24" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #25" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #26" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #27" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #28" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #29" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #30" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #31" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #32" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #33" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #34" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #35" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #36" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #37" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #38" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #39" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #40" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #41" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #42" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #43" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #44" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #45" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #46" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #47" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #48" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #49" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #50" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #51" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #52" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #53" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #54" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #55" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #56" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #57" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #58" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #59" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #60" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #61" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #62" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #63" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #64" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #65" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #66" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #67" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #68" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #69" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #70" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #71" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #72" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #73" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localdatetime #74" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select time into local date time #8" +Feature "DurationBetweenAcceptance": Scenario "Should compute negative duration between in big units #1" +Feature "DurationBetweenAcceptance": Scenario "Should compute negative duration between in big units #2" +Feature "DurationBetweenAcceptance": Scenario "Should compute negative duration between in big units #3" +Feature "DurationBetweenAcceptance": Scenario "Should compute negative duration between in big units #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute negative duration between in big units #5" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #1" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #2" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #3" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #5" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #6" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #7" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #8" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #9" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #10" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #11" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #12" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #13" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #14" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #15" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #16" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #17" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #18" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #19" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #20" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #21" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #22" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #23" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #24" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals #25" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #1" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #2" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #3" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #5" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #6" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #7" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #8" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #9" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #10" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #11" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #12" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #13" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #14" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #15" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #16" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #17" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #18" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #19" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #20" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in days #21" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from local date time #1" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from local date time #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from local date time #3" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #1" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #2" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #3" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #4" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #5" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #6" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #7" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #8" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #9" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #10" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #11" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #12" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #13" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #14" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #15" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #16" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #17" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #18" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #19" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #20" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #21" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #22" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #23" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #24" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #25" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #26" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #27" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #28" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #29" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #30" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #31" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #32" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #33" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #34" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #35" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #36" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #37" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #38" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #39" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #40" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #41" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #42" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #43" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #44" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #45" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #46" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #47" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #48" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #49" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #50" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate date #51" +Feature "TemporalCreateAcceptance": Scenario "Should store date time #1" +Feature "TemporalCreateAcceptance": Scenario "Should store date time #2" +Feature "TemporalCreateAcceptance": Scenario "Should store date time #3" +Feature "TemporalAccessorAcceptance": Scenario "Should provide accessors for duration" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #1" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #2" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #3" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #4" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #5" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #6" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #7" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #8" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #9" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #10" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #11" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #12" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #13" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #14" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #15" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #16" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #17" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #18" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #19" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #20" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #21" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #22" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #23" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #24" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #25" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #26" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #27" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #28" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #29" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #30" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #31" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #32" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #33" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #34" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #35" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #36" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #37" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #38" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #39" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #40" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #41" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #42" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #43" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #44" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #45" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #46" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #47" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #48" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #49" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #50" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #51" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #52" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #53" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #54" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #55" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #56" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #57" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #58" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #59" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #60" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #61" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #62" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #63" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #64" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #65" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #66" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #67" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #68" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #69" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #70" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #71" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #72" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #73" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #74" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #75" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #76" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #77" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #78" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #79" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #80" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #81" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #82" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #83" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #84" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #85" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #86" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #87" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #88" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #89" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #90" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #91" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #92" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #93" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #94" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #95" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #96" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #97" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #98" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #99" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #100" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #101" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #102" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #103" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #104" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate datetime #105" +Feature "TemporalCreateAcceptance": Scenario "Should store date #1" +Feature "TemporalCreateAcceptance": Scenario "Should store date #2" +Feature "TemporalCreateAcceptance": Scenario "Should store date #3" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #1" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #2" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #3" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #4" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #5" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #6" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #7" +Feature "TemporalParseAcceptance": Scenario "Should parse date time from string #8" +Feature "TemporalAccessorAcceptance": Scenario "Should provide accessors for date time" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #1" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #2" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #3" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #5" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #6" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #7" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #8" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #9" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #10" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #11" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #12" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #13" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #14" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #15" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #16" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #17" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #18" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #19" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #20" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between two temporals in months #21" +Feature "TemporalToStringAcceptance": Scenario "Should serialize time" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #1" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #2" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #3" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #4" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #5" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #6" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #7" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #8" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #9" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #10" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #11" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #12" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #13" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #14" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #15" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #16" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #17" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #18" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #19" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #20" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #21" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #22" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #23" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #24" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #25" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #26" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #27" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #28" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #29" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #30" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #31" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #32" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #33" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #34" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #35" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #36" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #37" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #38" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #39" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #40" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #41" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #42" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #43" +Feature "TemporalTruncateAcceptance": Scenario "Should truncate localtime #44" +Feature "DurationBetweenAcceptance": Scenario "Should split between boundaries correctly #1" +Feature "DurationBetweenAcceptance": Scenario "Should split between boundaries correctly #2" +Feature "DurationBetweenAcceptance": Scenario "Should split between boundaries correctly #3" +Feature "DurationBetweenAcceptance": Scenario "Should split between boundaries correctly #4" +Feature "DurationBetweenAcceptance": Scenario "Should split between boundaries correctly #5" +Feature "DurationBetweenAcceptance": Scenario "Should split between boundaries correctly #6" +Feature "DurationBetweenAcceptance": Scenario "Should compute duration between if they differ only by a fraction of a second and the first comes after the second." +Feature "TemporalCreateAcceptance": Scenario "Should construct local date time #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct local date time #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct local date time #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct local date time #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct local date time #17" +Feature "TemporalCreateAcceptance": Scenario "Should construct local date time #24" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #1" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #2" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #3" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #4" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #5" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #6" +Feature "TemporalParseAcceptance": Scenario "Should parse duration from string #7" +Feature "TemporalToStringAcceptance": Scenario "Should serialize date time" +Feature "TemporalToStringAcceptance": Scenario "Should serialize timezones correctly" +Feature "TemporalComparisonAcceptance": Scenario "Should compare times #1" +Feature "TemporalComparisonAcceptance": Scenario "Should compare times #2" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #1" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #2" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #3" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #4" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #5" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #6" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #7" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #8" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #9" +Feature "DurationBetweenAcceptance": Scenario "Should handle when seconds and subseconds have different signs #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct week localdatetime #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct week localdatetime #14" +Feature "TemporalCreateAcceptance": Scenario "Should construct week localdatetime #15" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #8" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #9" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #10" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #11" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #12" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #13" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #14" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #15" +Feature "TemporalSelectAcceptance": Scenario "Should select time into date time #16" +Feature "TemporalAccessorAcceptance": Scenario "Should provide accessors for local time" +Feature "TemporalSelectAcceptance": Scenario "Should select date #1" +Feature "TemporalSelectAcceptance": Scenario "Should select date #2" +Feature "TemporalSelectAcceptance": Scenario "Should select date #3" +Feature "TemporalSelectAcceptance": Scenario "Should select date #4" +Feature "TemporalSelectAcceptance": Scenario "Should select date #5" +Feature "TemporalSelectAcceptance": Scenario "Should select date #6" +Feature "TemporalSelectAcceptance": Scenario "Should select date #7" +Feature "TemporalSelectAcceptance": Scenario "Should select date #8" +Feature "TemporalSelectAcceptance": Scenario "Should select date #9" +Feature "TemporalSelectAcceptance": Scenario "Should select date #10" +Feature "TemporalSelectAcceptance": Scenario "Should select date #11" +Feature "TemporalSelectAcceptance": Scenario "Should select date #12" +Feature "TemporalSelectAcceptance": Scenario "Should select date #13" +Feature "TemporalSelectAcceptance": Scenario "Should select date #14" +Feature "TemporalSelectAcceptance": Scenario "Should select date #15" +Feature "TemporalSelectAcceptance": Scenario "Should select date #16" +Feature "TemporalSelectAcceptance": Scenario "Should select date #17" +Feature "TemporalSelectAcceptance": Scenario "Should select date #18" +Feature "TemporalSelectAcceptance": Scenario "Should select date #19" +Feature "TemporalSelectAcceptance": Scenario "Should select date #20" +Feature "TemporalSelectAcceptance": Scenario "Should select date #21" +Feature "DurationBetweenAcceptance": Scenario "Should compute durations with no difference #1" +Feature "DurationBetweenAcceptance": Scenario "Should compute durations with no difference #2" +Feature "DurationBetweenAcceptance": Scenario "Should compute durations with no difference #3" +Feature "DurationBetweenAcceptance": Scenario "Should compute durations with no difference #4" +Feature "DurationBetweenAcceptance": Scenario "Should compute durations with no difference #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #8" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #9" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #11" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #12" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #14" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #15" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #16" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #17" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #18" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #19" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #20" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #21" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #22" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #23" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #24" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #25" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #26" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #27" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #28" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with named time zone #29" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #6" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #7" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #8" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #9" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #10" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #11" +Feature "TemporalSelectAcceptance": Scenario "Should select local time #12" +Feature "TemporalSelectAcceptance": Scenario "Should select datetime into local date time #1" +Feature "TemporalSelectAcceptance": Scenario "Should select datetime into local date time #2" +Feature "TemporalSelectAcceptance": Scenario "Should select datetime into local date time #3" +Feature "TemporalSelectAcceptance": Scenario "Should select datetime into local date time #4" +Feature "TemporalSelectAcceptance": Scenario "Should select datetime into local date time #5" +Feature "TemporalSelectAcceptance": Scenario "Should select datetime into local date time #6" +Feature "TemporalComparisonAcceptance": Scenario "Should compare local times #1" +Feature "TemporalComparisonAcceptance": Scenario "Should compare local times #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #1" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #2" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #3" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #4" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #5" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #6" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #7" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #8" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #9" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #10" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #11" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #12" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #13" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #14" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #15" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #16" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #17" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #18" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #19" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #20" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #21" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #22" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #23" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #24" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #25" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #26" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #27" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #28" +Feature "TemporalCreateAcceptance": Scenario "Should construct date time with offset time zone #29" +Feature "TemporalCreateAcceptance": Scenario "Should store local date time array #1" +Feature "TemporalCreateAcceptance": Scenario "Should store local date time array #2" +Feature "TemporalCreateAcceptance": Scenario "Should store duration array #1" +Feature "TemporalCreateAcceptance": Scenario "Should store duration array #2" +Feature "TemporalCreateAcceptance": Scenario "Should store duration" +Feature "TemporalCreateAcceptance": Scenario "Should store date time array #1" +Feature "TemporalCreateAcceptance": Scenario "Should store date time array #2" +Feature "TemporalCreateAcceptance": Scenario "Should store time" +Feature "TemporalCreateAcceptance": Scenario "Should store date array #1" +Feature "TemporalCreateAcceptance": Scenario "Should store date array #2" +Feature "TemporalCreateAcceptance": Scenario "Should store local date time" +Feature "TemporalCreateAcceptance": Scenario "Should store local time" +Feature "TemporalCreateAcceptance": Scenario "Should store date time" +Feature "TemporalCreateAcceptance": Scenario "Should store date" +Feature "TemporalCreateAcceptance": Scenario "Should store local time array #1" +Feature "TemporalCreateAcceptance": Scenario "Should store local time array #2" +Feature "TemporalCreateAcceptance": Scenario "Should store time array #1" +Feature "TemporalCreateAcceptance": Scenario "Should store time array #2" +Feature "TemporalComparisonAcceptance": Scenario "Should compare local date times #1" +Feature "TemporalComparisonAcceptance": Scenario "Should compare local date times #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #1" \ No newline at end of file diff --git a/graph/cypher/src/test/resources/tck/wont_fix_blacklist b/graph/cypher/src/test/resources/tck/wont_fix_blacklist new file mode 100644 index 0000000000000..a53f6b1a9eb14 --- /dev/null +++ b/graph/cypher/src/test/resources/tck/wont_fix_blacklist @@ -0,0 +1,193 @@ +Feature "DeleteAcceptance": Scenario "Delete nodes" +Feature "DeleteAcceptance": Scenario "Detach delete node" +Feature "DeleteAcceptance": Scenario "Delete relationships" +Feature "DeleteAcceptance": Scenario "Deleting connected nodes" +Feature "DeleteAcceptance": Scenario "Detach deleting connected nodes and relationships" +Feature "DeleteAcceptance": Scenario "Detach deleting paths" +Feature "DeleteAcceptance": Scenario "Undirected expand followed by delete and count" +Feature "DeleteAcceptance": Scenario "Undirected variable length expand followed by delete and count" +Feature "DeleteAcceptance": Scenario "Create and delete in same query" +Feature "DeleteAcceptance": Scenario "Delete optionally matched relationship" +Feature "DeleteAcceptance": Scenario "Delete on null node" +Feature "DeleteAcceptance": Scenario "Detach delete on null node" +Feature "DeleteAcceptance": Scenario "Delete on null path" +Feature "DeleteAcceptance": Scenario "Delete node from a list" +Feature "DeleteAcceptance": Scenario "Delete relationship from a list" +Feature "DeleteAcceptance": Scenario "Delete nodes from a map" +Feature "DeleteAcceptance": Scenario "Delete relationships from a map" +Feature "DeleteAcceptance": Scenario "Detach delete nodes from nested map/list" +Feature "DeleteAcceptance": Scenario "Delete relationships from nested map/list" +Feature "DeleteAcceptance": Scenario "Delete paths from nested map/list" +Feature "DeleteAcceptance": Scenario "Delete relationship with bidirectional matching" +Feature "CreateAcceptance": Scenario "Create a single node with multiple labels" +Feature "CreateAcceptance": Scenario "Combine MATCH and CREATE" +Feature "CreateAcceptance": Scenario "Combine MATCH, WITH and CREATE" +Feature "CreateAcceptance": Scenario "Newly-created nodes not visible to preceding MATCH" +Feature "CreateAcceptance": Scenario "Create a single node with properties" +Feature "CreateAcceptance": Scenario "Creating a node with null properties should not return those properties" +Feature "CreateAcceptance": Scenario "Creating a relationship with null properties should not return those properties" +Feature "CreateAcceptance": Scenario "Create a simple pattern" +Feature "CreateAcceptance": Scenario "Create a self loop" +Feature "CreateAcceptance": Scenario "Create a self loop using MATCH" +Feature "CreateAcceptance": Scenario "Create nodes and relationships" +Feature "CreateAcceptance": Scenario "Create a relationship with a property" +Feature "CreateAcceptance": Scenario "Create a relationship with the correct direction" +Feature "CreateAcceptance": Scenario "Create a relationship and an end node from a matched starting node" +Feature "CreateAcceptance": Scenario "Create a single node after a WITH" +Feature "CreateAcceptance": Scenario "Create a relationship with a reversed direction" +Feature "CreateAcceptance": Scenario "Create a pattern with multiple hops" +Feature "CreateAcceptance": Scenario "Create a pattern with multiple hops in the reverse direction" +Feature "CreateAcceptance": Scenario "Create a pattern with multiple hops in varying directions" +Feature "CreateAcceptance": Scenario "Create a pattern with multiple hops with multiple types and varying directions" +Feature "CreateAcceptance": Scenario "Nodes are not created when aliases are applied to variable names" +Feature "CreateAcceptance": Scenario "Only a single node is created when an alias is applied to a variable name" +Feature "CreateAcceptance": Scenario "Nodes are not created when aliases are applied to variable names multiple times" +Feature "CreateAcceptance": Scenario "Only a single node is created when an alias is applied to a variable name multiple times" +Feature "CreateAcceptance": Scenario "A bound node should be recognized after projection with WITH + WITH" +Feature "CreateAcceptance": Scenario "A bound node should be recognized after projection with WITH + UNWIND" +Feature "CreateAcceptance": Scenario "A bound node should be recognized after projection with WITH + MERGE node" +Feature "CreateAcceptance": Scenario "A bound node should be recognized after projection with WITH + MERGE pattern" +Feature "CreateAcceptance": Scenario "Fail when trying to create using an undirected relationship pattern" +Feature "CreateAcceptance": Scenario "Creating a pattern with multiple hops and changing directions" +Feature "Create": Scenario "Creating a node" +Feature "Create": Scenario "Creating two nodes" +Feature "Create": Scenario "Creating two nodes and a relationship" +Feature "Create": Scenario "Creating a node with a label" +Feature "Create": Scenario "Creating a node with a property" +Feature "MergeRelationshipAcceptance": Scenario "Creating a relationship" +Feature "MergeRelationshipAcceptance": Scenario "Matching a relationship" +Feature "MergeRelationshipAcceptance": Scenario "Matching two relationships" +Feature "MergeRelationshipAcceptance": Scenario "Filtering relationships" +Feature "MergeRelationshipAcceptance": Scenario "Creating relationship when all matches filtered out" +Feature "MergeRelationshipAcceptance": Scenario "Matching incoming relationship" +Feature "MergeRelationshipAcceptance": Scenario "Creating relationship with property" +Feature "MergeRelationshipAcceptance": Scenario "Using ON CREATE on a node" +Feature "MergeRelationshipAcceptance": Scenario "Using ON CREATE on a relationship" +Feature "MergeRelationshipAcceptance": Scenario "Using ON MATCH on created node" +Feature "MergeRelationshipAcceptance": Scenario "Using ON MATCH on created relationship" +Feature "MergeRelationshipAcceptance": Scenario "Using ON MATCH on a relationship" +Feature "MergeRelationshipAcceptance": Scenario "Using ON CREATE and ON MATCH" +Feature "MergeRelationshipAcceptance": Scenario "Creating relationship using merged nodes" +Feature "MergeRelationshipAcceptance": Scenario "Mixing MERGE with CREATE" +Feature "MergeRelationshipAcceptance": Scenario "Introduce named paths 1" +Feature "MergeRelationshipAcceptance": Scenario "Introduce named paths 2" +Feature "MergeRelationshipAcceptance": Scenario "Use outgoing direction when unspecified" +Feature "MergeRelationshipAcceptance": Scenario "Match outgoing relationship when direction unspecified" +Feature "MergeRelationshipAcceptance": Scenario "Match both incoming and outgoing relationships when direction unspecified" +Feature "MergeRelationshipAcceptance": Scenario "Fail when imposing new predicates on a variable that is already bound" +Feature "MergeRelationshipAcceptance": Scenario "Using list properties via variable" +Feature "MergeRelationshipAcceptance": Scenario "Matching using list property" +Feature "MergeRelationshipAcceptance": Scenario "Using bound variables from other updating clause" +Feature "MergeRelationshipAcceptance": Scenario "UNWIND with multiple merges" +Feature "MergeRelationshipAcceptance": Scenario "Do not match on deleted entities" +Feature "MergeRelationshipAcceptance": Scenario "Do not match on deleted relationships" +Feature "MergeRelationshipAcceptance": Scenario "Aliasing of existing nodes 1" +Feature "MergeRelationshipAcceptance": Scenario "Aliasing of existing nodes 2" +Feature "MergeRelationshipAcceptance": Scenario "Double aliasing of existing nodes 1" +Feature "MergeRelationshipAcceptance": Scenario "Double aliasing of existing nodes 2" +Feature "MergeNodeAcceptance": Scenario "Merge node when no nodes exist" +Feature "MergeNodeAcceptance": Scenario "Merge node with label" +Feature "MergeNodeAcceptance": Scenario "Merge node with label add label on create" +Feature "MergeNodeAcceptance": Scenario "Merge node with label add property on create" +Feature "MergeNodeAcceptance": Scenario "Merge node with label when it exists" +Feature "MergeNodeAcceptance": Scenario "Merge node should create when it doesn't match, properties" +Feature "MergeNodeAcceptance": Scenario "Merge node should create when it doesn't match, properties and label" +Feature "MergeNodeAcceptance": Scenario "Merge node with prop and label" +Feature "MergeNodeAcceptance": Scenario "Merge node with label add label on match when it exists" +Feature "MergeNodeAcceptance": Scenario "Merge node with label add property on update when it exists" +Feature "MergeNodeAcceptance": Scenario "Merge node and set property on match" +Feature "MergeNodeAcceptance": Scenario "Should work when finding multiple elements" +Feature "MergeNodeAcceptance": Scenario "Should handle argument properly" +Feature "MergeNodeAcceptance": Scenario "Should handle arguments properly with only write clauses" +Feature "MergeNodeAcceptance": Scenario "Should be able to merge using property from match" +Feature "MergeNodeAcceptance": Scenario "Should be able to use properties from match in ON CREATE" +Feature "MergeNodeAcceptance": Scenario "Should be able to use properties from match in ON MATCH" +Feature "MergeNodeAcceptance": Scenario "Should be able to use properties from match in ON MATCH and ON CREATE" +Feature "MergeNodeAcceptance": Scenario "Should be able to set labels on match" +Feature "MergeNodeAcceptance": Scenario "Should be able to set labels on match and on create" +Feature "MergeNodeAcceptance": Scenario "Should support updates while merging" +Feature "MergeNodeAcceptance": Scenario "Merge must properly handle multiple labels" +Feature "MergeNodeAcceptance": Scenario "Merge followed by multiple creates" +Feature "MergeNodeAcceptance": Scenario "Unwind combined with merge" +Feature "MergeNodeAcceptance": Scenario "Merges should not be able to match on deleted nodes" +Feature "MergeNodeAcceptance": Scenario "ON CREATE on created nodes" +Feature "MergeIntoAcceptance": Scenario "Updating one property with ON CREATE" +Feature "MergeIntoAcceptance": Scenario "Null-setting one property with ON CREATE" +Feature "MergeIntoAcceptance": Scenario "Copying properties from node with ON CREATE" +Feature "MergeIntoAcceptance": Scenario "Copying properties from node with ON MATCH" +Feature "MergeIntoAcceptance": Scenario "Copying properties from literal map with ON CREATE" +Feature "MergeIntoAcceptance": Scenario "Copying properties from literal map with ON MATCH" +Feature "SetAcceptance": Scenario "Setting a node property to null removes the existing property" +Feature "SetAcceptance": Scenario "Setting a relationship property to null removes the existing property" +Feature "SetAcceptance": Scenario "Set a property" +Feature "SetAcceptance": Scenario "Set a property to an expression" +Feature "SetAcceptance": Scenario "Set a property by selecting the node using a simple expression" +Feature "SetAcceptance": Scenario "Set a property by selecting the relationship using a simple expression" +Feature "SetAcceptance": Scenario "Setting a property to null removes the property" +Feature "SetAcceptance": Scenario "Add a label to a node" +Feature "SetAcceptance": Scenario "Adding a list property" +Feature "SetAcceptance": Scenario "Concatenate elements onto a list property" +Feature "SetAcceptance": Scenario "Concatenate elements in reverse onto a list property" +Feature "SetAcceptance": Scenario "Overwrite values when using +=" +Feature "SetAcceptance": Scenario "Retain old values when using +=" +Feature "SetAcceptance": Scenario "Explicit null values in a map remove old values" +Feature "SetAcceptance": Scenario "Non-existent values in a property map are removed with SET =" +Feature "ReturnAcceptance2": Scenario "Fail when returning properties of deleted nodes" +Feature "ReturnAcceptance2": Scenario "Fail when returning labels of deleted nodes" +Feature "ReturnAcceptance2": Scenario "Fail when returning properties of deleted relationships" +Feature "ReturnAcceptance2": Scenario "Do not fail when returning type of deleted relationships" +Feature "ReturnAcceptance2": Scenario "Fail when sorting on variable removed by DISTINCT" +Feature "RemoveAcceptance": Scenario "Should ignore nulls" +Feature "RemoveAcceptance": Scenario "Remove a single label" +Feature "RemoveAcceptance": Scenario "Remove multiple labels" +Feature "RemoveAcceptance": Scenario "Remove a single node property" +Feature "RemoveAcceptance": Scenario "Remove multiple node properties" +Feature "RemoveAcceptance": Scenario "Remove a single relationship property" +Feature "RemoveAcceptance": Scenario "Remove multiple relationship properties" +Feature "RemoveAcceptance": Scenario "Remove a missing property should be a valid operation" +Feature "UnwindAcceptance": Scenario "Creating nodes from an unwound parameter list" +Feature "UnwindAcceptance": Scenario "Unwind with merge" +Feature "SemanticErrorAcceptance": Scenario "Failing when using variable length relationship in CREATE" +Feature "SemanticErrorAcceptance": Scenario "Failing when using variable length relationship in MERGE" +Feature "SemanticErrorAcceptance": Scenario "Failing when using parameter as node predicate in MERGE" +Feature "SemanticErrorAcceptance": Scenario "Failing when using parameter as relationship predicate in MERGE" +Feature "SemanticErrorAcceptance": Scenario "Failing when deleting an integer expression" +Feature "SemanticErrorAcceptance": Scenario "Failing when using CREATE on a node that is already bound" +Feature "SemanticErrorAcceptance": Scenario "Failing when using MERGE on a node that is already bound" +Feature "SemanticErrorAcceptance": Scenario "Failing when using CREATE on a relationship that is already bound" +Feature "SemanticErrorAcceptance": Scenario "Failing when using MERGE on a relationship that is already bound" +Feature "SemanticErrorAcceptance": Scenario "Failing when using undefined variable in ON CREATE" +Feature "NullAcceptance": Scenario "Ignore null when setting property" +Feature "NullAcceptance": Scenario "Ignore null when removing property" +Feature "NullAcceptance": Scenario "Ignore null when setting properties using an appending map" +Feature "NullAcceptance": Scenario "Ignore null when setting properties using an overriding map" +Feature "NullAcceptance": Scenario "Ignore null when setting label" +Feature "NullAcceptance": Scenario "Ignore null when removing label" +Feature "NullAcceptance": Scenario "Ignore null when deleting node" +Feature "NullAcceptance": Scenario "Ignore null when deleting relationship" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing on merging relationship with null property" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing on merging node with null property" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using undefined variable in SET" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using undefined variable in DELETE" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when using a variable that is already bound in CREATE" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when creating without direction" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when creating with two directions" +Feature "MiscellaneousErrorAcceptance": Scenario "Failing when deleting a label" +Feature "LargeCreateQuery": Scenario "Generate the movie graph correctly" +Feature "LargeCreateQuery": Scenario "Many CREATE clauses" +Feature "LabelsAcceptance": Scenario "Adding a single label" +Feature "LabelsAcceptance": Scenario "Ignore space before colon" +Feature "LabelsAcceptance": Scenario "Adding multiple labels" +Feature "LabelsAcceptance": Scenario "Ignoring intermediate whitespace 1" +Feature "LabelsAcceptance": Scenario "Ignoring intermediate whitespace 2" +Feature "LabelsAcceptance": Scenario "Creating node without label" +Feature "LabelsAcceptance": Scenario "Creating node with two labels" +Feature "LabelsAcceptance": Scenario "Ignore space when creating node with labels" +Feature "LabelsAcceptance": Scenario "Create node with label in pattern" +Feature "LabelsAcceptance": Scenario "Fail when adding a new label predicate on a node that is already bound 1" +Feature "LabelsAcceptance": Scenario "Fail when adding new label predicate on a node that is already bound 2" +Feature "LabelsAcceptance": Scenario "Fail when adding new label predicate on a node that is already bound 3" +Feature "LabelsAcceptance": Scenario "Fail when adding new label predicate on a node that is already bound 4" +Feature "LabelsAcceptance": Scenario "Fail when adding new label predicate on a node that is already bound 5" +Feature "LabelsAcceptance": Scenario "Removing a label" +Feature "LabelsAcceptance": Scenario "Removing a non-existent label" \ No newline at end of file diff --git a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala new file mode 100644 index 0000000000000..ed13540c0156d --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -0,0 +1,178 @@ +/* + * 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.spark.cypher + +import org.apache.spark.SparkFunSuite +import org.apache.spark.graph.api.{CypherResult, NodeDataset, PropertyGraph, RelationshipDataset} +import org.apache.spark.sql.{Dataset, Row, SaveMode} + +class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { + + test("create PropertyGraph from single NodeDataset") { + val nodeData: Dataset[Row] = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") + val nodeDataset: NodeDataset = NodeDataset.builder(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + val graph: PropertyGraph = cypherSession.createGraph(Array(nodeDataset), Array.empty[RelationshipDataset]) + val result: CypherResult = graph.cypher("MATCH (n) RETURN n") + result.ds.show() + } + + test("create PropertyGraph from Node- and RelationshipDatasets") { + val nodeData: Dataset[Row] = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") + val relationshipData: Dataset[Row] = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") + val nodeDataset: NodeDataset = NodeDataset.builder(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + val relationshipDataset: RelationshipDataset = RelationshipDataset.builder(relationshipData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .build() + val graph: PropertyGraph = cypherSession.createGraph(Array(nodeDataset), Array(relationshipDataset)) + val result: CypherResult = graph.cypher( + """ + |MATCH (a:Person)-[r:KNOWS]->(:Person) + |RETURN a, r""".stripMargin) + result.ds.show() + } + + test("create PropertyGraph with multiple node and relationship types") { + val studentDS: Dataset[Row] = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") + val teacherDS: Dataset[Row] = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") + + val studentNF = NodeDataset.builder(studentDS) + .idColumn("id") + .labelSet(Array("Person", "Student")) + .properties(Map("name" -> "name", "age" -> "age")) + .build() + + val teacherNF = NodeDataset.builder(teacherDS) + .idColumn("id") + .labelSet(Array("Person", "Teacher")) + .properties(Map("name" -> "name", "subject" -> "subject")) + .build() + + val knowsDF: Dataset[Row] = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") + val teachesDF: Dataset[Row] = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") + + val knowsRF: RelationshipDataset = RelationshipDataset.builder(knowsDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() + val teachesRF: RelationshipDataset = RelationshipDataset.builder(teachesDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("TEACHES") + .build() + + val graph: PropertyGraph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) + val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") + result.ds.show() + } + + test("create PropertyGraph with multiple node and relationship types and explicit property-to-column mappings") { + val studentDF: Dataset[Row] = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "col_name", "col_age") + val teacherDF: Dataset[Row] = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "col_name", "col_subject") + + val studentNF: NodeDataset = NodeDataset(studentDF, "id", Set("Person", "Student"), Map("name" -> "col_name", "age" -> "col_age")) + val teacherNF: NodeDataset = NodeDataset(teacherDF, "id", Set("Person", "Teacher"), Map("name" -> "col_name", "subject" -> "col_subject")) + + val knowsDF: Dataset[Row] = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "col_since") + val teachesDF: Dataset[Row] = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") + + val knowsRF: RelationshipDataset = RelationshipDataset(knowsDF, "id", "source", "target", "KNOWS", Map("since" -> "col_since")) + val teachesRF: RelationshipDataset = RelationshipDataset(teachesDF, "id", "source", "target", "TEACHES", Map.empty) + + val graph: PropertyGraph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) + val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") + result.ds.show() + } + + test("create PropertyGraph with multiple node and relationship types stored in wide tables") { + val nodeDF: Dataset[Row] = spark.createDataFrame(Seq( + (0L, true, true, false, Some("Alice"), Some(42), None), + (1L, true, true, false, Some("Bob"), Some(23), None), + (2L, true, false, true, Some("Eve"), None, Some("CS")), + )).toDF("$ID", ":Person", ":Student", ":Teacher", "name", "age", "subject") + + val relsDF: Dataset[Row] = spark.createDataFrame(Seq( + (0L, 0L, 1L, true, false, Some(1984)), + (1L, 2L, 1L, false, true, None) + )).toDF("$ID", "$SOURCE_ID", "$TARGET_ID", ":KNOWS", ":TEACHES", "since") + + val graph: PropertyGraph = cypherSession.createGraph(nodeDF, relsDF) + val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") + result.ds.show() + } + + test("save and load PropertyGraph") { + val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) + graph1.nodes.show() + graph1.write.mode(SaveMode.Overwrite).save("/tmp/my-storage") + val graph2: PropertyGraph = cypherSession.read.load("/tmp/my-storage") + graph2.nodes.show() + } + + test("round trip example using column name conventions") { + val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) + val graph2: PropertyGraph = cypherSession.createGraph(graph1.nodes, graph1.relationships) + graph2.nodes.show() + graph2.relationships.show() + } + + test("example for retaining user ids") { + val nodesWithRetainedId = nodes.withColumn("retainedId", nodes.col("$ID")) + val relsWithRetainedId = relationships.withColumn("retainedId", relationships.col("$ID")) + + cypherSession + .createGraph(nodesWithRetainedId, relsWithRetainedId) + .cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").ds.show() + } + + lazy val nodes: Dataset[Row] = spark.createDataFrame(Seq( + (0L, true, false, Some("Alice"), Some(42), None), + (1L, true, false, Some("Bob"), Some(23), None), + (2L, true, false, Some("Carol"), Some(22), None), + (3L, true, false, Some("Eve"), Some(19), None), + (4L, false, true, None, None, Some("UC Berkeley")), + (5L, false, true, None, None, Some("Stanford")) + )).toDF("$ID", ":Student", ":University", "name", "age", "title") + + lazy val relationships: Dataset[Row] = spark.createDataFrame(Seq( + (0L, 0L, 1L, true, false), + (1L, 0L, 3L, true, false), + (2L, 1L, 3L, true, false), + (3L, 3L, 0L, true, false), + (4L, 3L, 1L, true, false), + (5L, 0L, 4L, false, true), + (6L, 1L, 4L, false, true), + (7L, 3L, 4L, false, true), + (8L, 2L, 5L, false, true), + )).toDF("$ID", "$SOURCE_ID", "$TARGET_ID", ":KNOWS", ":STUDY_AT") +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala new file mode 100644 index 0000000000000..702ea28c9b116 --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -0,0 +1,95 @@ +/* + * 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.spark.cypher + +import java.nio.file.Paths + +import org.apache.spark.graph.api.{NodeDataset, RelationshipDataset} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} +import org.junit.rules.TemporaryFolder +import org.scalatest.BeforeAndAfterEach + +class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with BeforeAndAfterEach { + + private var tempDir: TemporaryFolder = _ + + override def beforeEach(): Unit = { + tempDir = new TemporaryFolder() + tempDir.create() + super.beforeEach() + } + + override def afterEach(): Unit = { + super.afterEach() + tempDir.delete() + } + + private def basePath: String = s"file://${Paths.get(tempDir.getRoot.getAbsolutePath)}" + + private lazy val nodeData: Dataset[Row] = spark.createDataFrame(Seq( + 0 -> "Alice", + 1 -> "Bob" + )).toDF("id", "name") + + private lazy val relationshipData: Dataset[Row] = spark.createDataFrame(Seq( + Tuple3(0, 0, 1) + )).toDF("id", "source", "target") + + private lazy val nodeDataFrame: NodeDataset = NodeDataset.builder(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + + private lazy val relationshipFrame: RelationshipDataset = RelationshipDataset(relationshipData, "id", "source", "target", "KNOWS", Map.empty) + + test("save and load a graph") { + val graph = cypherSession.createGraph(Array(nodeDataFrame), Array(relationshipFrame)) + graph.write.save(basePath) + + val readGraph = cypherSession.read.load(basePath) + readGraph.cypher( + "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" + ).ds.show() + } + + test("save and loads a property graph") { + val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") + val nodeFrame = NodeDataset.builder(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + + val relationshipData = spark + .createDataFrame(Seq((0L, 0L, 1L, 1984))) + .toDF("id", "source", "target", "since") + val relationshipFrame = RelationshipDataset(relationshipData, "id", "source", "target", "KNOWS", Map.empty) + + val writeGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) + + withTempDir(file => { + writeGraph.write.mode(SaveMode.Overwrite).save(file.getAbsolutePath) + val readGraph = cypherSession.read.load(file.getAbsolutePath) + + checkAnswer(readGraph.nodes, writeGraph.nodes) + checkAnswer(readGraph.relationships, writeGraph.relationships) + }) + } +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala new file mode 100644 index 0000000000000..1abe0ee6aed39 --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -0,0 +1,321 @@ +/* + * 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.spark.cypher + +import org.scalatest.Matchers + +import org.apache.spark.graph.api.{CypherSession, NodeDataset, RelationshipDataset} +import org.apache.spark.graph.api.CypherSession._ +import org.apache.spark.sql.{Dataset, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.NumberConverter +import org.apache.spark.sql.test.SharedSparkSession + +class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers { + + type IdType = Array[Byte] + + def convertId(inputId: Long): IdType = NumberConverter.toBinary(inputId) + + /** + * This method is supposed to return an implementation of [[CypherSession]] + * from a module that depends on spark-graph-api (e.g. spark-cypher). + * + * This allows us to run the same tests on arbitrary implementations. + */ + def cypherSession: CypherSession = SparkCypherSession.create + + lazy val nodes: Dataset[Row] = spark + .createDataFrame( + Seq( + (0L, true, true, false, false, Some(42), Some("Alice"), None, None), + (1L, true, true, false, false, Some(23), Some("Bob"), None, None), + (2L, true, false, true, false, Some(22), Some("Carol"), Some("CS"), None), + (3L, true, true, false, false, Some(19), Some("Eve"), None, None), + (4L, false, false, false, true, None, None, None, Some("UC Berkeley")), + (5L, false, false, false, true, None, None, None, Some("Stanford")))) + .toDF( + ID_COLUMN, + label("Person"), + label("Student"), + label("Teacher"), + label("University"), + "age", + "name", + "subject", + "title") + + lazy val relationships: Dataset[Row] = spark + .createDataFrame( + Seq( + (0L, 0L, 1L, true, false), + (1L, 0L, 3L, true, false), + (2L, 1L, 3L, true, false), + (3L, 3L, 0L, true, false), + (4L, 3L, 1L, true, false), + (5L, 0L, 4L, false, true), + (6L, 1L, 4L, false, true), + (7L, 3L, 4L, false, true), + (8L, 2L, 5L, false, true))) + .toDF(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN, label("KNOWS"), label("STUDY_AT")) + + test("create graph from NodeDataset") { + val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") + val nodeDataset = NodeDataset.builder(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + val graph = cypherSession.createGraph(Array(nodeDataset), Array.empty[RelationshipDataset]) + + val expectedDf = spark + .createDataFrame(Seq((convertId(0L), true, "Alice"), (convertId(1L), true, "Bob"))) + .toDF(ID_COLUMN, label("Person"), "name") + + checkAnswer(graph.nodes, expectedDf) + } + + test("create graph from NodeDataset and RelationshipDataset") { + val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") + val nodeDataset = NodeDataset.builder(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + val relationshipData = spark + .createDataFrame(Seq((0L, 0L, 1L, 1984))) + .toDF("id", "source", "target", "since") + val relationshipDataset = RelationshipDataset.builder(relationshipData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() + + val graph = cypherSession.createGraph(Array(nodeDataset), Array(relationshipDataset)) + + val expectedNodeDf = spark + .createDataFrame(Seq((convertId(0L), true, "Alice"), (convertId(1L), true, "Bob"))) + .toDF(ID_COLUMN, label("Person"), "name") + + val expectedRelDf = spark + .createDataFrame(Seq((convertId(0L), convertId(0L), convertId(1L), true, 1984))) + .toDF(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN, label("KNOWS"), "since") + + checkAnswer(graph.nodes, expectedNodeDf) + checkAnswer(graph.relationships, expectedRelDf) + } + + test("create graph with multiple node and relationship types") { + val studentDF = spark + .createDataFrame(Seq((0L, "Alice", 42), (1L, "Bob", 23))) + .toDF("id", "name", "age") + val teacherDF = spark + .createDataFrame(Seq((2L, "Eve", "CS"))) + .toDF("id", "name", "subject") + + val studentNodeDataset = NodeDataset.builder(studentDF) + .idColumn("id") + .labelSet(Array("Person", "Student")) + .properties(Map("name" -> "name", "age" -> "age")) + .build() + + val teacherNodeDataset = NodeDataset.builder(teacherDF) + .idColumn("id") + .labelSet(Array("Person", "Teacher")) + .properties(Map("name" -> "name", "subject" -> "subject")) + .build() + + val knowsDF = spark + .createDataFrame(Seq((0L, 0L, 1L, 1984))) + .toDF("id", "source", "target", "since") + val teachesDF = spark + .createDataFrame(Seq((1L, 2L, 1L))) + .toDF("id", "source", "target") + + val knowsRelationshipDataset = RelationshipDataset.builder(knowsDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() + val teachesRelationshipDataset = RelationshipDataset.builder(teachesDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("TEACHES") + .build() + + val graph = cypherSession.createGraph( + Array(studentNodeDataset, teacherNodeDataset), + Array(knowsRelationshipDataset, teachesRelationshipDataset)) + + val expectedNodeDf = spark + .createDataFrame( + Seq( + (convertId(0L), true, true, false, Some(42), Some("Alice"), None), + (convertId(1L), true, true, false, Some(23), Some("Bob"), None), + (convertId(2L), true, false, true, None, Some("Eve"), Some("CS")) + )) + .toDF( + ID_COLUMN, + label("Person"), + label("Student"), + label("Teacher"), + "age", + "name", + "subject") + + val expectedRelDf = spark + .createDataFrame( + Seq( + (convertId(0L), convertId(0L), convertId(1L), true, false, Some(1984)), + (convertId(1L), convertId(2L), convertId(1L), false, true, None))) + .toDF( + ID_COLUMN, + SOURCE_ID_COLUMN, + TARGET_ID_COLUMN, + label("KNOWS"), + label("TEACHES"), + "since") + + checkAnswer(graph.nodes, expectedNodeDf) + checkAnswer(graph.relationships, expectedRelDf) + } + + test("create graph with explicit property-to-column mappings") { + val studentDF = spark + .createDataFrame(Seq((0L, "Alice", 42), (1L, "Bob", 23))) + .toDF("id", "col_name", "col_age") + val teacherDF = spark + .createDataFrame(Seq((2L, "Eve", "CS"))) + .toDF("id", "col_name", "col_subject") + + val studentNodeDataset = NodeDataset( + studentDF, + "id", + Set("Person", "Student"), + propertyColumns = Map("name" -> "col_name", "age" -> "col_age")) + val teacherNodeDataset = NodeDataset( + teacherDF, + "id", + Set("Person", "Teacher"), + propertyColumns = Map("name" -> "col_name", "subject" -> "col_subject")) + + val knowsDF = spark.createDataFrame(Seq((0L, 0L, 1L, 1984))).toDF("id", "source", "target", "col_since") + val teachesDF = spark.createDataFrame(Seq((1L, 2L, 1L))).toDF("id", "source", "target") + + val knowsRelationshipDataset = RelationshipDataset( + knowsDF, + "id", + "source", + "target", + relationshipType = "KNOWS", + propertyColumns = Map("since" -> "col_since")) + val teachesRelationshipDataset = RelationshipDataset( + teachesDF, + "id", + "source", + "target", + "TEACHES", + Map.empty) + + val graph = cypherSession.createGraph( + Array(studentNodeDataset, teacherNodeDataset), + Array(knowsRelationshipDataset, teachesRelationshipDataset)) + + val expectedNodeDf = spark + .createDataFrame( + Seq( + (convertId(0L), true, true, false, Some(42), Some("Alice"), None), + (convertId(1L), true, true, false, Some(23), Some("Bob"), None), + (convertId(2L), true, false, true, None, Some("Eve"), Some("CS")) + )) + .toDF( + ID_COLUMN, + label("Person"), + label("Student"), + label("Teacher"), + "age", + "name", + "subject") + + val expectedRelDf = spark + .createDataFrame( + Seq( + (convertId(0L), convertId(0L), convertId(1L), true, false, Some(1984)), + (convertId(1L), convertId(2L), convertId(1L), false, true, None))) + .toDF( + ID_COLUMN, + SOURCE_ID_COLUMN, + TARGET_ID_COLUMN, + label("KNOWS"), + label("TEACHES"), + "since") + + checkAnswer(graph.nodes, expectedNodeDf) + checkAnswer(graph.relationships, expectedRelDf) + } + + test("select nodes via label set") { + val graph = cypherSession.createGraph(nodes, relationships) + val nodeDataset = graph.nodeDataset(Array("Person", "Teacher")) + + nodeDataset.labelSet shouldEqual Set("Person", "Teacher") + nodeDataset.idColumn shouldEqual ID_COLUMN + nodeDataset.propertyColumns shouldEqual Map( + "age" -> "age", + "name" -> "name", + "subject" -> "subject", + "title" -> "title") + + val expectedNodeDf = spark + .createDataFrame(Seq((convertId(2L), Some(22), Some("Carol"), Some("CS"), None: Option[String]))) + .toDF(ID_COLUMN, "age", "name", "subject", "title") + + checkAnswer(nodeDataset.ds, expectedNodeDf) + } + + test("select relationships via type") { + val graph = cypherSession.createGraph(nodes, relationships) + val relationshipDataset = graph.relationshipDataset("KNOWS") + + relationshipDataset.relationshipType shouldEqual "KNOWS" + relationshipDataset.idColumn shouldEqual ID_COLUMN + relationshipDataset.sourceIdColumn shouldEqual SOURCE_ID_COLUMN + relationshipDataset.targetIdColumn shouldEqual TARGET_ID_COLUMN + relationshipDataset.propertyColumns shouldBe empty + + val expectedRelDf = spark + .createDataFrame( + Seq( + (convertId(0L), convertId(0L), convertId(1L)), + (convertId(1L), convertId(0L), convertId(3L)), + (convertId(2L), convertId(1L), convertId(3L)), + (convertId(3L), convertId(3L), convertId(0L)), + (convertId(4L), convertId(3L), convertId(1L)))) + .toDF(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN) + + checkAnswer(relationshipDataset.ds, expectedRelDf) + } + + private def label(label: String): String = s"$LABEL_COLUMN_PREFIX$label" + +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala new file mode 100644 index 0000000000000..992058a2c053f --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala @@ -0,0 +1,48 @@ +/* + * 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.spark.cypher + +import org.apache.spark.SparkConf +import org.apache.spark.graph.api.CypherSession +import org.apache.spark.sql.test.SharedSparkSession +import org.scalatest.Suite + +trait SharedCypherContext extends SharedSparkSession { + self: Suite => + + private var _cypherEngine: SparkCypherSession = _ + + protected implicit def cypherSession: CypherSession = _cypherEngine + + def internalCypherSession: SparkCypherSession = _cypherEngine + + override protected def sparkConf: SparkConf = super.sparkConf + // Required for left outer join without join expressions in OPTIONAL MATCH (leads to cartesian product) + .set("spark.sql.crossJoin.enabled", "true") + + override def beforeAll() { + super.beforeAll() + _cypherEngine = SparkCypherSession.createInternal + } + + protected override def afterAll(): Unit = { + _cypherEngine = null + super.afterAll() + } +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala new file mode 100644 index 0000000000000..9b375f89abb62 --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala @@ -0,0 +1,153 @@ +/* + * 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.spark.cypher.construction + +import java.time.{LocalDate, LocalDateTime} + +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.conversions.TemporalConversions._ +import org.apache.spark.cypher.conversions.TypeConversions._ +import org.apache.spark.cypher.{SparkCypherSession, SparkEntityTable} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Dataset, Row} +import org.opencypher.okapi.api.graph.Pattern +import org.opencypher.okapi.api.io.conversion.{ElementMapping, NodeMappingBuilder, RelationshipMappingBuilder} +import org.opencypher.okapi.api.schema.PropertyKeys.PropertyKeys +import org.opencypher.okapi.impl.exception.IllegalArgumentException +import org.opencypher.okapi.impl.temporal.Duration +import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph +import org.opencypher.okapi.relational.impl.graph.ScanGraph +import org.opencypher.okapi.testing.propertygraph.{CreateGraphFactory, CypherTestGraphFactory, InMemoryTestGraph} + +import scala.collection.JavaConverters._ + +object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { + + def encodeIdColumns(df: Dataset[Row], mapping: ElementMapping): Dataset[Row] = { + val idCols = mapping.allSourceIdKeys.map { columnName => + val dataType = df.schema.fields(df.schema.fieldIndex(columnName)).dataType + dataType match { + case LongType => df.col(columnName).cast(StringType).cast(BinaryType) + case IntegerType => df.col(columnName).cast(StringType).cast(BinaryType) + case StringType => df.col(columnName).cast(BinaryType) + case BinaryType => df.col(columnName) + case unsupportedType => throw IllegalArgumentException( + expected = s"Column `$columnName` should have a valid identifier data type, such as [`$BinaryType`, `$StringType`, `$LongType`, `$IntegerType`]", + actual = s"Unsupported column type `$unsupportedType`" + ) + } + } + val remainingCols = mapping.allSourceKeys.filterNot(mapping.allSourceIdKeys.contains).map(df.col) + val colsToSelect = idCols ++ remainingCols + df.select(colsToSelect: _*) + } + + + def initGraph(createQuery: String) + (implicit sparkCypher: SparkCypherSession): RelationalCypherGraph[DataFrameTable] = { + apply(CreateGraphFactory(createQuery)) + } + + val tableEntityIdKey = "___id" + val tableEntityStartNodeKey = "___source" + val tableEntityEndNodeKey = "___target" + + override def apply(propertyGraph: InMemoryTestGraph, additionalPattern: Seq[Pattern] = Seq.empty) + (implicit sparkCypher: SparkCypherSession): ScanGraph[DataFrameTable] = { + require(additionalPattern.isEmpty, "Additional pattern input not yet supported.") + val schema = computeSchema(propertyGraph) + + val nodeScans = schema.labelCombinations.combos.map { labels => + val propKeys = schema.nodePropertyKeys(labels) + + val idStructField = Seq(StructField(tableEntityIdKey, LongType, nullable = false)) + val structType = StructType(idStructField ++ getPropertyStructFields(propKeys)) + + val header = Seq(tableEntityIdKey) ++ propKeys.keys + val rows = propertyGraph.nodes + .filter(_.labels == labels) + .map { node => + val propertyValues = propKeys.map(key => + node.properties.unwrap.get(key._1) match { + case Some(date: LocalDate) => java.sql.Date.valueOf(date) + case Some(localDateTime: LocalDateTime) => java.sql.Timestamp.valueOf(localDateTime) + case Some(dur: Duration) => dur.toCalendarInterval + case Some(other) => other + case None => null + } + ) + Row.fromSeq(Seq(node.id) ++ propertyValues) + } + + val records = sparkCypher.sparkSession.createDataFrame(rows.asJava, structType).toDF(header: _*) + + val nodeMapping = NodeMappingBuilder + .on(tableEntityIdKey) + .withImpliedLabels(labels.toSeq: _*) + .withPropertyKeys(propKeys.keys.toSeq: _*) + .build + + val encodedRecords = encodeIdColumns(records, nodeMapping) + + SparkEntityTable(nodeMapping, encodedRecords) + } + + val relScans = schema.relationshipTypes.map { relType => + val propKeys = schema.relationshipPropertyKeys(relType) + + val idStructFields = Seq( + StructField(tableEntityIdKey, LongType, nullable = false), + StructField(tableEntityStartNodeKey, LongType, nullable = false), + StructField(tableEntityEndNodeKey, LongType, nullable = false)) + val structType = StructType(idStructFields ++ getPropertyStructFields(propKeys)) + + val header = Seq(tableEntityIdKey, tableEntityStartNodeKey, tableEntityEndNodeKey) ++ propKeys.keys + val rows = propertyGraph.relationships + .filter(_.relType == relType) + .map { rel => + val propertyValues = propKeys.map(key => rel.properties.unwrap.getOrElse(key._1, null)) + Row.fromSeq(Seq(rel.id, rel.startId, rel.endId) ++ propertyValues) + } + + val records = sparkCypher.sparkSession.createDataFrame(rows.asJava, structType).toDF(header: _*) + + val relationshipMapping = RelationshipMappingBuilder + .on(tableEntityIdKey) + .from(tableEntityStartNodeKey) + .to(tableEntityEndNodeKey) + .relType(relType) + .withPropertyKeys(propKeys.keys.toSeq: _*) + .build + + val encodedRecords = encodeIdColumns(records, relationshipMapping) + + SparkEntityTable(relationshipMapping, encodedRecords) + } + + new ScanGraph(nodeScans.toSeq ++ relScans, schema) + } + + override def name: String = getClass.getSimpleName + + protected def getPropertyStructFields(propKeys: PropertyKeys): Seq[StructField] = { + propKeys.foldLeft(Seq.empty[StructField]) { + case (fields, key) => fields :+ StructField(key._1, key._2.getSparkType, key._2.isNullable) + } + } +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/tck/SparkCypherTckSuite.scala new file mode 100644 index 0000000000000..9901f1c3846fc --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/tck/SparkCypherTckSuite.scala @@ -0,0 +1,127 @@ +/* + * 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.spark.cypher.tck + +import java.io.File + +import org.apache.spark.SparkFunSuite +import org.apache.spark.cypher.construction.ScanGraphFactory +import org.apache.spark.cypher.{SharedCypherContext, SparkCypherSession} +import org.opencypher.okapi.tck.test.Tags.{BlackList, WhiteList} +import org.opencypher.okapi.tck.test.{ScenariosFor, TCKGraph} +import org.opencypher.okapi.testing.propertygraph.CypherTestGraphFactory +import org.opencypher.tools.tck.api.CypherTCK +import org.scalatest.Tag +import org.scalatest.prop.TableDrivenPropertyChecks._ + +import scala.io.Source +import scala.util.{Failure, Success, Try} + +class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { + + private val tckSparkCypherTag = Tag("TckSparkCypher") + + private val graphFactory: CypherTestGraphFactory[SparkCypherSession] = ScanGraphFactory + + private val failingBlacklist = getClass.getResource("/tck/failing_blacklist").getFile + private val temporalBlacklist = getClass.getResource("/tck/temporal_blacklist").getFile + private val wontFixBlacklist = getClass.getResource("/tck/wont_fix_blacklist").getFile + private val failureReportingBlacklist = getClass.getResource("/tck/failure_reporting_blacklist").getFile + private val scenarios = ScenariosFor(failingBlacklist, temporalBlacklist, wontFixBlacklist, failureReportingBlacklist) + + forAll(scenarios.whiteList) { scenario => + test(s"[${WhiteList.name}] $scenario", WhiteList, tckSparkCypherTag, Tag(graphFactory.name)) { + scenario(TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession)).execute() + } + } + + forAll(scenarios.blackList) { scenario => + test(s"[${graphFactory.name}, ${BlackList.name}] $scenario", BlackList, tckSparkCypherTag) { + val tckGraph = TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession) + + Try(scenario(tckGraph).execute()) match { + case Success(_) => + throw new RuntimeException(s"A blacklisted scenario passed: $scenario") + case Failure(_) => + } + } + } + + test("compute TCK coverage") { + def withSource[T](s: Source)(f: Source => T) = try { f(s) } finally { s.close() } + + val failingScenarios = withSource(Source.fromFile(failingBlacklist))(_.getLines().size) + val failingTemporalScenarios = withSource(Source.fromFile(temporalBlacklist))(_.getLines().size) + val failureReportingScenarios = withSource(Source.fromFile(failureReportingBlacklist))(_.getLines().size) + + val white = scenarios.whiteList.groupBy(_.featureName).mapValues(_.size) + val black = scenarios.blackList.groupBy(_.featureName).mapValues(_.size) + + val allFeatures = white.keySet ++ black.keySet + val perFeatureCoverage = allFeatures.foldLeft(Map.empty[String, Float]) { + case (acc, feature) => + val w = white.getOrElse(feature, 0).toFloat + val b = black.getOrElse(feature, 0).toFloat + val percentage = (w / (w + b)) * 100 + acc.updated(feature, percentage) + } + + val allScenarios = scenarios.blacklist.size + scenarios.whiteList.size.toFloat + val readOnlyScenarios = scenarios.whiteList.size + failingScenarios + failureReportingScenarios.toFloat + failingTemporalScenarios + val smallReadOnlyScenarios = scenarios.whiteList.size + failingScenarios.toFloat + + val overallCoverage = scenarios.whiteList.size / allScenarios + val readOnlyCoverage = scenarios.whiteList.size / readOnlyScenarios + val smallReadOnlyCoverage = scenarios.whiteList.size / smallReadOnlyScenarios + + val featureCoverageReport = + perFeatureCoverage.map { case (feature, coverage) => s" $feature: $coverage%" }.mkString("\n") + + val report = + s"""|TCK Coverage + |------------ + | + | Complete: ${overallCoverage * 100}% + | Read Only: ${readOnlyCoverage * 100}% + | Read Only (without Failure case Scenarios and temporal): ${smallReadOnlyCoverage * 100}% + | + |Feature Coverage + |---------------- + | + |$featureCoverageReport + """.stripMargin + + println(report) + + } + + ignore("run custom scenario") { + val file = new File(getClass.getResource("CustomTest.feature").toURI) + + CypherTCK + .parseFilesystemFeature(file) + .scenarios + .foreach(scenario => scenario(TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession)).execute()) + } + + ignore("run single scenario") { + scenarios.get("Should add or subtract duration to or from date") + .foreach(scenario => scenario(TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession)).execute()) + } +} diff --git a/pom.xml b/pom.xml index 9c2aa9de85ce6..5b6994ca6d722 100644 --- a/pom.xml +++ b/pom.xml @@ -2992,7 +2992,7 @@ scala-2.12 - + scala-2.13 diff --git a/repl/src/main/scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala b/repl/src/main/scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 34d2f45e715e9..f7e7061cb1e07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -456,7 +456,7 @@ object CatalystTypeConverters { case d: JavaBigDecimal => new DecimalConverter(DecimalType(d.precision, d.scale)).toCatalyst(d) case seq: Seq[Any] => new GenericArrayData(seq.map(convertToCatalyst).toArray) case r: Row => InternalRow(r.toSeq.map(convertToCatalyst): _*) - case arr: Array[Any] => new GenericArrayData(arr.map(convertToCatalyst)) + case arr: Array[_] => new GenericArrayData(arr.map(convertToCatalyst)) case map: Map[_, _] => ArrayBasedMapData( map,