From c93f1a15f0b9c14065fd7f851857a5be2a9a24b5 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 19 Feb 2019 13:37:37 +0100 Subject: [PATCH 001/123] Add initial set of classes to get Cypher engine running Co-authored-by: Philip Stutz --- graph/cypher/pom.xml | 5 + .../graph/cypher/SparkCypherEntity.scala | 54 ++ .../graph/cypher/SparkCypherRecords.scala | 115 +++++ .../graph/cypher/SparkCypherSession.scala | 30 ++ .../spark/graph/cypher/SparkNodeTable.scala | 16 + .../spark/graph/cypher/SparkTable.scala | 123 +++++ .../conversions/CypherValueEncoders.scala | 36 ++ .../cypher/conversions/ExprConversions.scala | 485 ++++++++++++++++++ .../cypher/conversions/RowConversion.scala | 133 +++++ .../conversions/TemporalConversions.scala | 116 +++++ .../cypher/conversions/TypeConversions.scala | 182 +++++++ .../spark/graph/cypher/udfs/LegacyUdfs.scala | 86 ++++ .../graph/cypher/udfs/TemporalUdfs.scala | 137 +++++ .../cypher/SparkCypherSessionSuite.scala | 23 + 14 files changed, 1541 insertions(+) create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala create mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 12b3832d9777a..2cf235b42980d 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -36,6 +36,11 @@ http://spark.apache.org/ + + org.opencypher + okapi-relational + ${okapi.version} + org.apache.spark spark-core_${scala.binary.version} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala new file mode 100644 index 0000000000000..36c054fe4ffd5 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala @@ -0,0 +1,54 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.cypher.SparkCypherEntity._ +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherNode, CypherRelationship} + +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 CypherNode[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 CypherRelationship[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/graph/cypher/SparkCypherRecords.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala new file mode 100644 index 0000000000000..7560444c014ee --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala @@ -0,0 +1,115 @@ +package org.apache.spark.graph.cypher + +import java.util.Collections + +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.graph.cypher.conversions.RowConversion +import org.apache.spark.graph.cypher.conversions.TypeConversions._ +import org.apache.spark.graph.cypher.conversions.CypherValueEncoders._ +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.EntityTable +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 fromEntityTable(entityTable: EntityTable[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) + } + + // /** + // * Wraps a Spark SQL table (DataFrame) in a CAPSRecords, making it understandable by Cypher. + // * + // * @param df table to wrap. + // * @param caps session to which the resulting CAPSRecords is tied. + // * @return a Cypher table. + // */ + // private[spark] def wrap(df: DataFrame)(implicit caps: SparkCypherSession): CAPSRecords = { + // val compatibleDf = df.withCypherCompatibleTypes + // CAPSRecords(compatibleDf.schema.toRecordHeader, compatibleDf) + // } + + 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 df: DataFrame = table.df + + override def cache(): SparkCypherRecords = { + df.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/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala new file mode 100644 index 0000000000000..cc616806bfb27 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -0,0 +1,30 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.sql.SparkSession +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.RelationalEntityTableFactory + +object SparkCypherSession { + def create(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) +} + +class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] { + + override type Result = RelationalCypherResult[DataFrameTable] + override type Records = SparkCypherRecords + override type Graph = RelationalCypherGraph[DataFrameTable] + + 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 entityTables: RelationalEntityTableFactory[DataFrameTable] = ??? +} + + diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala new file mode 100644 index 0000000000000..9b9eaa6aba14a --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala @@ -0,0 +1,16 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.opencypher.okapi.api.io.conversion.NodeMapping +import org.opencypher.okapi.relational.api.io.NodeTable + +case class SparkNodeTable(override val mapping: NodeMapping, override val table: DataFrameTable) + extends NodeTable(mapping, table) with RecordBehaviour { + + override type Records = SparkNodeTable + + override def cache(): SparkNodeTable = { + table.cache() + this + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala new file mode 100644 index 0000000000000..16c0fd10fedb3 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -0,0 +1,123 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.cypher.conversions.ExprConversions._ +import org.apache.spark.graph.cypher.conversions.TypeConversions._ +import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.{Column, DataFrame} +import org.opencypher.okapi.api.types.CypherType +import org.opencypher.okapi.api.value.CypherValue +import org.opencypher.okapi.api.value.CypherValue.CypherMap +import org.opencypher.okapi.impl.exception.IllegalArgumentException +import org.opencypher.okapi.ir.api.expr.{Aggregator, Expr, Var} +import org.opencypher.okapi.relational.api.table.Table +import org.opencypher.okapi.relational.impl.planning.{JoinType, Order} +import org.opencypher.okapi.relational.impl.table.RecordHeader + +object SparkTable { + + implicit class DataFrameTable(val df: DataFrame) extends Table[DataFrameTable] { + + override def columnType: Map[String, CypherType] = physicalColumns.map(c => c -> df.cypherTypeForColumn(c)).toMap + + override def physicalColumns: Seq[String] = df.columns + + override def select(cols: String*): DataFrameTable = { + if (df.columns.toSeq == cols) { + df + } else { + df.select(cols.map(df.col): _*) + } + } + + 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 withColumnsRenamed(columnRenamings: Map[String, String]): DataFrameTable = { + df.safeRenameColumns(columnRenamings) + } + + override def filter(expr: Expr)(implicit header: RecordHeader, parameters: CypherValue.CypherMap): DataFrameTable = ??? + override def drop(cols: String*): DataFrameTable = ??? + override def join( + other: DataFrameTable, + joinType: JoinType, + joinCols: (String, String)* + ): DataFrameTable = ??? + override def unionAll(other: DataFrameTable): DataFrameTable = ??? + override def orderBy(sortItems: (Expr, Order)*) + ( + implicit header: RecordHeader, + parameters: CypherValue.CypherMap + ): DataFrameTable = ??? + override def skip(n: Long): DataFrameTable = ??? + override def limit(n: Long): DataFrameTable = ??? + override def distinct: DataFrameTable = ??? + override def group( + by: Set[Var], + aggregations: Set[(Aggregator, (String, CypherType))] + ) + ( + implicit header: RecordHeader, + parameters: CypherValue.CypherMap + ): DataFrameTable = ??? + + + override def show(rows: Int): Unit = ??? + + override def columnsFor(returnItem: String): Set[String] = ??? + override def rows: Iterator[String => CypherValue.CypherValue] = ??? + override def size: Long = ??? + } + + implicit class DataFrameOps(val df: DataFrame) extends AnyVal { + + 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(renamings: Map[String, String]): DataFrame = { + if (renamings.isEmpty || renamings.forall { case (oldColumn, newColumn) => oldColumn == newColumn }) { + df + } else { + renamings.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 renamings.contains(col) => renamings(col) + case col => col + } + df.toDF(newColumns: _*) + } + } + + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala new file mode 100644 index 0000000000000..fd0502a1a6fe1 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala @@ -0,0 +1,36 @@ +package org.apache.spark.graph.cypher.conversions + +import org.apache.spark.graph.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].asInstanceOf[ExpressionEncoder[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/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala new file mode 100644 index 0000000000000..7cc9db47d1f91 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -0,0 +1,485 @@ +package org.apache.spark.graph.cypher.conversions + +import org.apache.spark.graph.cypher.conversions.TemporalConversions._ +import org.apache.spark.graph.cypher.conversions.TypeConversions._ +import org.apache.spark.graph.cypher.udfs.LegacyUdfs._ +import org.apache.spark.graph.cypher.udfs.TemporalUdfs +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Column, DataFrame, functions} +import org.opencypher.okapi.api.types._ +import org.opencypher.okapi.api.value.CypherValue.{CypherList, CypherMap} +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, IllegalStateException, NotImplementedException, UnsupportedOperationException} +import org.opencypher.okapi.impl.temporal.TemporalTypesHelper._ +import org.opencypher.okapi.impl.temporal.{Duration => DurationValue} +import org.opencypher.okapi.ir.api.PropertyKey +import org.opencypher.okapi.ir.api.expr._ +import org.opencypher.okapi.relational.impl.table.RecordHeader + +object ExprConversions { + + private val NULL_LIT: Column = functions.lit(null) + + private val TRUE_LIT: Column = functions.lit(true) + + private val FALSE_LIT: Column = functions.lit(false) + + private val ONE_LIT: Column = functions.lit(1) + + private val E: Column = functions.lit(Math.E) + + private val PI: Column = functions.lit(Math.PI) + + implicit class RichExpression(expr: Expr) { + + def verify(implicit header: RecordHeader): Unit = { + if (header.expressionsFor(expr).isEmpty) throw IllegalStateException(s"Expression $expr not in header:\n${header.pretty}") + } + + /** + * This is possible without violating Cypher semantics because + * - Spark SQL returns null when comparing across types (from initial investigation) + * - We never have multiple types per column in CAPS (yet) + */ + def compare(comparator: Column => Column => Column, lhs: Expr, rhs: Expr) + (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { + comparator(lhs.asSparkSQLExpr)(rhs.asSparkSQLExpr) + } + + def lt(c: Column): Column => Column = c < _ + + def lteq(c: Column): Column => Column = c <= _ + + def gt(c: Column): Column => Column = c > _ + + def gteq(c: Column): Column => Column = c >= _ + + /** + * 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 df the dataframe 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, df: DataFrame, parameters: CypherMap): Column = { + + expr match { + + // context based lookups + case p@Param(name) if p.cypherType.isInstanceOf[CTList] => + parameters(name) match { + case CypherList(l) => functions.array(l.unwrap.map(functions.lit): _*) + case notAList => throw IllegalArgumentException("a Cypher list", notAList) + } + + case Param(name) => + toSparkLiteral(parameters(name).unwrap) + + case Property(e, PropertyKey(key)) => + e.cypherType.material match { + case CTMap(inner) => + if (inner.keySet.contains(key)) e.asSparkSQLExpr.getField(key) else functions.lit(null) + + case CTDate => + TemporalConversions.temporalAccessor[Date](e.asSparkSQLExpr, key) + + case CTLocalDateTime => + TemporalConversions.temporalAccessor[Timestamp](e.asSparkSQLExpr, key) + + case CTDuration => + TemporalUdfs.durationAccessor(key.toLowerCase).apply(e.asSparkSQLExpr) + + case _ if !header.contains(expr) || !df.columns.contains(header.column(expr)) => + NULL_LIT + + case _ => + verify + + df.col(header.column(expr)) + } + + // direct column lookup + case _: Var | _: Param | _: HasLabel | _: HasType | _: StartNode | _: EndNode => + verify + + val colName = header.column(expr) + if (df.columns.contains(colName)) { + df.col(colName) + } else { + NULL_LIT + } + + case AliasExpr(innerExpr, _) => + innerExpr.asSparkSQLExpr + + // Literals + case ListLit(exprs) => + functions.array(exprs.map(_.asSparkSQLExpr): _*) + + case NullLit(ct) => + NULL_LIT.cast(ct.toSparkType.get) + + case LocalDateTime(dateExpr) => + dateExpr match { + case Some(e) => + val localDateTimeValue = TemporalConversions.resolveTemporalArgument(e) + .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 + + functions.lit(localDateTimeValue).cast(DataTypes.TimestampType) + case None => functions.current_timestamp() + } + + case Date(dateExpr) => + dateExpr match { + case Some(e) => + val dateValue = TemporalConversions.resolveTemporalArgument(e) + .map(parseDate) + .map(java.sql.Date.valueOf) + .orNull + + functions.lit(dateValue).cast(DataTypes.DateType) + case None => functions.current_timestamp() + } + + case Duration(durationExpr) => + val durationValue = TemporalConversions.resolveTemporalArgument(durationExpr).map { + case Left(m) => DurationValue(m.mapValues(_.toLong)).toCalendarInterval + case Right(s) => DurationValue.parse(s).toCalendarInterval + }.orNull + functions.lit(durationValue) + + case l: Lit[_] => functions.lit(l.v) + + // predicates + case Equals(e1, e2) => e1.asSparkSQLExpr === e2.asSparkSQLExpr + case Not(e) => !e.asSparkSQLExpr + case IsNull(e) => e.asSparkSQLExpr.isNull + case IsNotNull(e) => e.asSparkSQLExpr.isNotNull + case Size(e) => + val col = e.asSparkSQLExpr + e.cypherType match { + case CTString => functions.length(col).cast(LongType) + case _: CTList | _: CTListOrNull => + functions.when( + col.isNotNull, + functions.size(col).cast(LongType) + ) + case CTNull => NULL_LIT + case other => throw NotImplementedException(s"size() on values of type $other") + } + + case Ands(exprs) => + exprs.map(_.asSparkSQLExpr).foldLeft(TRUE_LIT)(_ && _) + + case Ors(exprs) => + exprs.map(_.asSparkSQLExpr).foldLeft(FALSE_LIT)(_ || _) + + case In(lhs, rhs) => + if (rhs.cypherType == CTNull || lhs.cypherType == CTNull) { + NULL_LIT.cast(BooleanType) + } else { + val element = lhs.asSparkSQLExpr + val array = rhs.asSparkSQLExpr + array_contains(array, element) + } + + case LessThan(lhs, rhs) => compare(lt, lhs, rhs) + case LessThanOrEqual(lhs, rhs) => compare(lteq, lhs, rhs) + case GreaterThanOrEqual(lhs, rhs) => compare(gteq, lhs, rhs) + case GreaterThan(lhs, rhs) => compare(gt, lhs, rhs) + + case StartsWith(lhs, rhs) => + lhs.asSparkSQLExpr.startsWith(rhs.asSparkSQLExpr) + case EndsWith(lhs, rhs) => + lhs.asSparkSQLExpr.endsWith(rhs.asSparkSQLExpr) + case Contains(lhs, rhs) => + lhs.asSparkSQLExpr.contains(rhs.asSparkSQLExpr) + + case RegexMatch(prop, Param(name)) => + val regex: String = parameters(name).unwrap.toString + prop.asSparkSQLExpr.rlike(regex) + + // Arithmetics + case Add(lhs, rhs) => + val lhsCT = lhs.cypherType + val rhsCT = rhs.cypherType + lhsCT.material -> rhsCT.material match { + case (_: CTList, _) => + throw UnsupportedOperationException("List concatenation is not supported") + + case (_, _: CTList) => + throw UnsupportedOperationException("List concatenation is not supported") + + case (CTString, _) if rhsCT.subTypeOf(CTNumber).maybeTrue => + functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr.cast(StringType)) + + case (_, CTString) if lhsCT.subTypeOf(CTNumber).maybeTrue => + functions.concat(lhs.asSparkSQLExpr.cast(StringType), rhs.asSparkSQLExpr) + + case (CTString, CTString) => + functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + + case (CTDate, CTDuration) => + TemporalUdfs.dateAdd(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + + case _ => + lhs.asSparkSQLExpr + rhs.asSparkSQLExpr + } + + case Subtract(lhs, rhs) if lhs.cypherType.material.subTypeOf(CTDate).isTrue && rhs.cypherType.material.subTypeOf(CTDuration).isTrue => + TemporalUdfs.dateSubtract(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + + case Subtract(lhs, rhs) => + lhs.asSparkSQLExpr - rhs.asSparkSQLExpr + + case Multiply(lhs, rhs) => lhs.asSparkSQLExpr * rhs.asSparkSQLExpr + case div@Divide(lhs, rhs) => (lhs.asSparkSQLExpr / rhs.asSparkSQLExpr).cast(div.cypherType.getSparkType) + + // Id functions + + case Id(e) => e.asSparkSQLExpr + + case PrefixId(idExpr, prefix) => +// idExpr.asSparkSQLExpr.addPrefix(functions.lit(prefix)) + ??? + + case ToId(e) => + e.cypherType.material match { + // TODO: Remove this call; we shouldn't have nodes or rels as concrete types here + case _: CTNode | _: CTRelationship => + e.asSparkSQLExpr + case CTInteger => +// e.asSparkSQLExpr.encodeLongAsCAPSId + ??? + case CTIdentity => + e.asSparkSQLExpr + case other => + throw IllegalArgumentException("a type that may be converted to an ID", other) + } + + // Functions + case _: MonotonicallyIncreasingId => functions.monotonically_increasing_id() + case Exists(e) => e.asSparkSQLExpr.isNotNull + case Labels(e) => + e.cypherType match { + case _: CTNode | _: CTNodeOrNull => + val node = e.owner.get + val labelExprs = header.labelsFor(node) + val (labelNames, labelColumns) = labelExprs + .toSeq + .map(e => e.label.name -> e.asSparkSQLExpr) + .sortBy(_._1) + .unzip + val booleanLabelFlagColumn = functions.array(labelColumns: _*) + get_node_labels(labelNames)(booleanLabelFlagColumn) + case CTNull => NULL_LIT + case other => throw IllegalArgumentException("an expression with type CTNode, CTNodeOrNull, or CTNull", other) + } + + case Keys(e) => e.cypherType.material match { + case _: CTNode | _: CTRelationship => + val node = e.owner.get + val propertyExprs = header.propertiesFor(node).toSeq.sortBy(_.key.name) + val (propertyKeys, propertyColumns) = propertyExprs.map(e => e.key.name -> e.asSparkSQLExpr).unzip + val valuesColumn = functions.array(propertyColumns: _*) + get_property_keys(propertyKeys)(valuesColumn) + + case CTMap(innerTypes) => + val mapColumn = e.asSparkSQLExpr + val (keys, valueColumns) = innerTypes.keys.map { e => + // Whe have to make sure that every column has the same type (true or null) + e -> functions.when(mapColumn.getField(e).isNotNull, functions.lit(true)).otherwise(NULL_LIT) + }.toSeq.unzip + val valueColumn = functions.array(valueColumns: _*) + get_property_keys(keys)(valueColumn) + + case other => throw IllegalArgumentException("an Expression with type CTNode, CTRelationship or CTMap", other) + } + + case Properties(e) => + e.cypherType.material match { + case _: CTNode | _: CTRelationship => + val element = e.owner.get + val propertyExprs = header.propertiesFor(element).toSeq.sortBy(_.key.name) + val propertyColumns = propertyExprs.map(e => e.asSparkSQLExpr.as(e.key.name)) + createStructColumn(propertyColumns) + case _: CTMap => e.asSparkSQLExpr + case other => + throw IllegalArgumentException("a node, relationship or map", other, "Invalid input to properties function") + } + + case Type(inner) => + inner match { + case v: Var => + val typeExprs = header.typesFor(v) + val (relTypeNames, relTypeColumn) = typeExprs.toSeq.map(e => e.relType.name -> e.asSparkSQLExpr).unzip + val booleanLabelFlagColumn = functions.array(relTypeColumn: _*) + get_rel_type(relTypeNames)(booleanLabelFlagColumn) + case _ => + throw NotImplementedException(s"Inner expression $inner of $expr is not yet supported (only variables)") + } + + case StartNodeFunction(e) => + val rel = e.owner.get + header.startNodeFor(rel).asSparkSQLExpr + + case EndNodeFunction(e) => + val rel = e.owner.get + header.endNodeFor(rel).asSparkSQLExpr + + case ToFloat(e) => e.asSparkSQLExpr.cast(DoubleType) + + case ToInteger(e) => e.asSparkSQLExpr.cast(IntegerType) + + case ToString(e) => e.asSparkSQLExpr.cast(StringType) + + case ToBoolean(e) => e.asSparkSQLExpr.cast(BooleanType) + + case Explode(list) => list.cypherType match { + case CTList(_) | CTListOrNull(_) => functions.explode(list.asSparkSQLExpr) + case CTNull => functions.explode(functions.lit(null).cast(ArrayType(NullType))) + case other => throw IllegalArgumentException("CTList", other) + } + + case Trim(str) => functions.trim(str.asSparkSQLExpr) + case LTrim(str) => functions.ltrim(str.asSparkSQLExpr) + case RTrim(str) => functions.rtrim(str.asSparkSQLExpr) + + case ToUpper(str) => functions.upper(str.asSparkSQLExpr) + case ToLower(str) => functions.lower(str.asSparkSQLExpr) + + case Range(from, to, maybeStep) => + val stepCol = maybeStep.map(_.asSparkSQLExpr).getOrElse(ONE_LIT) + rangeUdf(from.asSparkSQLExpr, to.asSparkSQLExpr, stepCol) + + case Replace(original, search, replacement) => translateColumn(original.asSparkSQLExpr, search.asSparkSQLExpr, replacement.asSparkSQLExpr) + + case Substring(original, start, maybeLength) => + val origCol = original.asSparkSQLExpr + val startCol = start.asSparkSQLExpr + ONE_LIT + val lengthCol = maybeLength.map(_.asSparkSQLExpr).getOrElse(functions.length(origCol) - startCol + ONE_LIT) + origCol.substr(startCol, lengthCol) + + // Mathematical functions + + case _: E => E + case _: Pi => PI + + case Sqrt(e) => functions.sqrt(e.asSparkSQLExpr) + case Log(e) => functions.log(e.asSparkSQLExpr) + case Log10(e) => functions.log(10.0, e.asSparkSQLExpr) + case Exp(e) => functions.exp(e.asSparkSQLExpr) + case Abs(e) => functions.abs(e.asSparkSQLExpr) + case Ceil(e) => functions.ceil(e.asSparkSQLExpr).cast(DoubleType) + case Floor(e) => functions.floor(e.asSparkSQLExpr).cast(DoubleType) + case _: Rand => functions.rand() + case Round(e) => functions.round(e.asSparkSQLExpr).cast(DoubleType) + case Sign(e) => functions.signum(e.asSparkSQLExpr).cast(IntegerType) + + case Acos(e) => functions.acos(e.asSparkSQLExpr) + case Asin(e) => functions.asin(e.asSparkSQLExpr) + case Atan(e) => functions.atan(e.asSparkSQLExpr) + case Atan2(e1, e2) => functions.atan2(e1.asSparkSQLExpr, e2.asSparkSQLExpr) + case Cos(e) => functions.cos(e.asSparkSQLExpr) + case Cot(e) => Divide(IntegerLit(1)(CTInteger), Tan(e)(CTFloat))(CTFloat).asSparkSQLExpr + case Degrees(e) => functions.degrees(e.asSparkSQLExpr) + case Haversin(e) => Divide(Subtract(IntegerLit(1)(CTInteger), Cos(e)(CTFloat))(CTFloat), IntegerLit(2)(CTInteger))(CTFloat).asSparkSQLExpr + case Radians(e) => functions.radians(e.asSparkSQLExpr) + case Sin(e) => functions.sin(e.asSparkSQLExpr) + case Tan(e) => functions.tan(e.asSparkSQLExpr) + + + // Time functions + + case Timestamp() => functions.current_timestamp().cast(LongType) + + // Bit operations + + case BitwiseAnd(lhs, rhs) => + lhs.asSparkSQLExpr.bitwiseAND(rhs.asSparkSQLExpr) + + case BitwiseOr(lhs, rhs) => + lhs.asSparkSQLExpr.bitwiseOR(rhs.asSparkSQLExpr) + + case ShiftLeft(value, IntegerLit(shiftBits)) => + functions.shiftLeft(value.asSparkSQLExpr, shiftBits.toInt) + + case ShiftRightUnsigned(value, IntegerLit(shiftBits)) => + functions.shiftRightUnsigned(value.asSparkSQLExpr, shiftBits.toInt) + + // Pattern Predicate + case ep: ExistsPatternExpr => ep.targetField.asSparkSQLExpr + + case Coalesce(es) => + val columns = es.map(_.asSparkSQLExpr) + functions.coalesce(columns: _*) + + case c: CaseExpr => + val alternatives = c.alternatives.map { + case (predicate, action) => functions.when(predicate.asSparkSQLExpr, action.asSparkSQLExpr) + } + + val alternativesWithDefault = c.default match { + case Some(inner) => alternatives :+ inner.asSparkSQLExpr + case None => alternatives + } + + val reversedColumns = alternativesWithDefault.reverse + + val caseColumn = reversedColumns.tail.foldLeft(reversedColumns.head) { + case (tmpCol, whenCol) => whenCol.otherwise(tmpCol) + } + caseColumn + + case ContainerIndex(container, index) => + val indexCol = index.asSparkSQLExpr + val containerCol = container.asSparkSQLExpr + + container.cypherType.material match { + case _: CTList | _: CTMap => containerCol.get(indexCol) + case other => throw NotImplementedException(s"Accessing $other by index is not supported") + } + + case MapExpression(items) => expr.cypherType.material match { + case CTMap(inner) => + val innerColumns = items.map { + case (key, innerExpr) => + val targetType = inner(key).toSparkType.get + innerExpr.asSparkSQLExpr.cast(targetType).as(key) + }.toSeq + createStructColumn(innerColumns) + case other => throw IllegalArgumentException("an expression of type CTMap", other) + } + + + case _ => + throw NotImplementedException(s"No support for converting Cypher expression $expr to a Spark SQL expression") + } + } + } + + private def toSparkLiteral(value: Any): Column = value match { + case map: Map[_, _] => + val columns = map.map { + case (key, v) => toSparkLiteral(v).as(key.toString) + }.toSeq + createStructColumn(columns) + case _ => functions.lit(value) + } + + private def createStructColumn(structColumns: Seq[Column]): Column = { + if (structColumns.isEmpty) { + functions.lit(null).cast(new StructType()) + } else { + functions.struct(structColumns: _*) + } + } + +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala new file mode 100644 index 0000000000000..29825a3b367f3 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala @@ -0,0 +1,133 @@ +package org.apache.spark.graph.cypher.conversions + +import org.apache.spark.graph.cypher.conversions.TemporalConversions._ +import org.apache.spark.graph.cypher.{SparkCypherNode, SparkCypherRelationship} +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.unsafe.types.CalendarInterval +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 _: CTNode => + collectNode(row, v) + + case _: CTRelationship => + collectRel(row, v) + + case CTList(_) if !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)) + raw match { + case interval: CalendarInterval => interval.toDuration + case other => CypherValue(other) + } + } + } + + 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/graph/cypher/conversions/TemporalConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala new file mode 100644 index 0000000000000..310a4c92e0d73 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala @@ -0,0 +1,116 @@ +package org.apache.spark.graph.cypher.conversions + +import java.time.temporal.ChronoUnit + +import org.apache.spark.graph.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, NotImplementedException, UnsupportedOperationException} +import org.opencypher.okapi.impl.temporal.{Duration, TemporalConstants} +import org.opencypher.okapi.ir.api.expr.{Expr, MapExpression, NullLit, Param} + +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) + } + } + + def resolveTemporalArgument(expr: Expr) + (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/graph/cypher/conversions/TypeConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala new file mode 100644 index 0000000000000..994e42a46f387 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala @@ -0,0 +1,182 @@ +package org.apache.spark.graph.cypher.conversions + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.types._ +import org.opencypher.okapi.api.types._ +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 { + + // Spark data types that are supported within the Cypher type system + val supportedTypes: Seq[DataType] = Seq( + // numeric + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + // other + StringType, + BooleanType, + DateType, + TimestampType, + NullType + ) + + 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 | CTVoid => Some(NullType) + case _ => + ct.material match { + case CTString => Some(StringType) + case CTInteger => Some(LongType) + case CTBoolean => Some(BooleanType) + case CTFloat => Some(DoubleType) + case CTLocalDateTime => Some(TimestampType) + case CTDate => Some(DateType) + case CTDuration => Some(CalendarIntervalType) + case CTIdentity => Some(BinaryType) + case _: CTNode => Some(BinaryType) + case _: CTRelationship => Some(BinaryType) + case CTList(CTVoid) => Some(ArrayType(NullType, containsNull = true)) + case CTList(CTNull) => Some(ArrayType(NullType, containsNull = true)) + case CTList(elemType) => + elemType.toSparkType.map(ArrayType(_, elemType.isNullable)) + case CTMap(inner) => + val innerFields = inner.map { + case (key, valueType) => valueType.toStructField(key) + }.toSeq + Some(StructType(innerFields)) + case _ => + None + } + } + + def getSparkType: DataType = toSparkType match { + case Some(t) => t + case None => throw NotImplementedException(s"Mapping of CypherType $ct to Spark type") + } + + def isSparkCompatible: Boolean = toSparkType.isDefined + + } + + 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 TimestampType => Some(CTLocalDateTime) + case DateType => Some(CTDate) + case CalendarIntervalType => Some(CTDuration) + case ArrayType(NullType, _) => Some(CTList(CTVoid)) + 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 = dt match { + case ArrayType(internalType, _) => internalType.isCypherCompatible + case StructType(fields) => fields.forall(_.dataType.isCypherCompatible) + case other => supportedTypes.contains(other) + } + + /** + * 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(_ && _) + } + +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala new file mode 100644 index 0000000000000..0eef092b38dc4 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala @@ -0,0 +1,86 @@ +package org.apache.spark.graph.cypher.udfs + +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.expressions.{ArrayContains, StringTranslate, XxHash64} +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions.{monotonically_increasing_id, udf} +import org.apache.spark.sql.types.{ArrayType, StringType} +import org.apache.spark.sql.{Column, functions} + +object LegacyUdfs { + + 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)) + } + + val rangeUdf: UserDefinedFunction = + udf[Array[Int], Int, Int, Int]((from: Int, to: Int, step: Int) => from.to(to, step).toArray) + + private[spark] val rowIdSpaceBitsUsedByMonotonicallyIncreasingId = 33 + + /** + * Configurable wrapper around `monotonically_increasing_id` + * + * @param partitionStartDelta Conceptually this number is added to the `partitionIndex` from which the Spark function + * `monotonically_increasing_id` starts assigning IDs. + */ + // TODO: Document inherited limitations with regard to the maximum number of rows per data frame + // TODO: Document the maximum number of partitions (before entering tag space) + def partitioned_id_assignment(partitionStartDelta: Int): Column = + monotonically_increasing_id() + (partitionStartDelta.toLong << rowIdSpaceBitsUsedByMonotonicallyIncreasingId) + + /** + * 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 array_append_long(array: Column, value: Column): Column = + appendLongUDF(array, value) + + private val appendLongUDF = + functions.udf(appendLong _) + + private def appendLong(array: Seq[Long], element: Long): Seq[Long] = + array :+ element + + def get_rel_type(relTypeNames: Seq[String]): UserDefinedFunction = { + val extractRelTypes = (booleanMask: Seq[Boolean]) => filterWithMask(relTypeNames)(booleanMask) + functions.udf(extractRelTypes.andThen(_.headOption.orNull), StringType) + } + + def get_node_labels(labelNames: Seq[String]): UserDefinedFunction = + functions.udf(filterWithMask(labelNames) _, ArrayType(StringType, containsNull = false)) + + private def filterWithMask(dataToFilter: Seq[String])(mask: Seq[Boolean]): Seq[String] = + dataToFilter.zip(mask).collect { + case (label, true) => label + } + + def get_property_keys(propertyKeys: Seq[String]): UserDefinedFunction = + functions.udf(filterNotNull(propertyKeys) _, ArrayType(StringType, containsNull = false)) + + private def filterNotNull(dataToFilter: Seq[String])(values: Seq[Any]): Seq[String] = + dataToFilter.zip(values).collect { + case (key, value) if value != null => key + } + + /** + * Alternative version of {{{org.apache.spark.sql.functions.translate}}} that takes {{{org.apache.spark.sql.Column}}}s for search and replace strings. + */ + def translateColumn(src: Column, matchingString: Column, replaceString: Column): Column = { + new Column(StringTranslate(src.expr, matchingString.expr, replaceString.expr)) + } + +} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala new file mode 100644 index 0000000000000..9aba46794276a --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala @@ -0,0 +1,137 @@ +package org.apache.spark.graph.cypher.udfs + +import java.sql.{Date, Timestamp} +import java.time.temporal.{ChronoField, IsoFields, TemporalField} + +import org.apache.logging.log4j.scala.Logging +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 extends Logging { + + /** + * 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 + + if (interval.microseconds % CalendarInterval.MICROS_PER_DAY != 0) { + logger.warn("Arithmetic with Date and Duration can lead to incorrect results when sub-day values are present.") + } + + 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 + + if (interval.microseconds % CalendarInterval.MICROS_PER_DAY != 0) { + logger.warn("Arithmetic with Date and Duration can lead to incorrect results when sub-day values are present.") + } + + 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/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala new file mode 100644 index 0000000000000..08c4943621cd9 --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala @@ -0,0 +1,23 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.sql.SparkSession +import org.apache.spark.{LocalSparkContext, SparkFunSuite} +import org.opencypher.okapi.api.io.conversion.NodeMapping + +class SparkCypherSessionSuite extends SparkFunSuite with LocalSparkContext { + + test("Initialize SparkCypherSession") { + implicit val sparkSession: SparkSession = SparkSession.builder().master("local[*]").getOrCreate() + implicit val sparkCypherSession: SparkCypherSession = SparkCypherSession.create + + val nodesDf = sparkSession.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val nodesMapping = NodeMapping.on("id").withImpliedLabel("Person").withPropertyKey("name") + + val sparkNodeTable = SparkNodeTable(nodesMapping, nodesDf) + + val graph = sparkCypherSession.readFrom(sparkNodeTable) + + graph.cypher("MATCH (n) RETURN n").show + } + +} From 3d5f69943678f9509cb1ffe0ae3e0b54a0514b59 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 19 Feb 2019 14:01:14 +0100 Subject: [PATCH 002/123] Add initial test infrastructure Co-authored-by: Philip Stutz --- .../spark/graph/cypher/BasicMatchSuite.scala | 19 +++++++++++++++ .../graph/cypher/SharedCypherContext.scala | 20 ++++++++++++++++ .../cypher/SparkCypherSessionSuite.scala | 23 ------------------- 3 files changed, 39 insertions(+), 23 deletions(-) create mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala create mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala delete mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala new file mode 100644 index 0000000000000..bc20e05068eb9 --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -0,0 +1,19 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.SparkFunSuite +import org.opencypher.okapi.api.io.conversion.NodeMapping + +class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { + + test("match single node pattern") { + val nodesDf = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val nodesMapping = NodeMapping.on("id").withImpliedLabel("Person").withPropertyKey("name") + + val sparkNodeTable = SparkNodeTable(nodesMapping, nodesDf) + + val graph = sparkCypher.readFrom(sparkNodeTable) + + graph.cypher("MATCH (n) RETURN n").show + } + +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala new file mode 100644 index 0000000000000..5ba8b9ca3d67e --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala @@ -0,0 +1,20 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.sql.test.SharedSQLContext + +trait SharedCypherContext extends SharedSQLContext { + + private var _sparkCypher: SparkCypherSession = _ + + protected implicit def sparkCypher: SparkCypherSession = _sparkCypher + + override def beforeAll() { + super.beforeAll() + _sparkCypher = SparkCypherSession.create + } + + protected override def afterAll(): Unit = { + _sparkCypher = null + super.afterAll() + } +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala deleted file mode 100644 index 08c4943621cd9..0000000000000 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SparkCypherSessionSuite.scala +++ /dev/null @@ -1,23 +0,0 @@ -package org.apache.spark.graph.cypher - -import org.apache.spark.sql.SparkSession -import org.apache.spark.{LocalSparkContext, SparkFunSuite} -import org.opencypher.okapi.api.io.conversion.NodeMapping - -class SparkCypherSessionSuite extends SparkFunSuite with LocalSparkContext { - - test("Initialize SparkCypherSession") { - implicit val sparkSession: SparkSession = SparkSession.builder().master("local[*]").getOrCreate() - implicit val sparkCypherSession: SparkCypherSession = SparkCypherSession.create - - val nodesDf = sparkSession.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") - val nodesMapping = NodeMapping.on("id").withImpliedLabel("Person").withPropertyKey("name") - - val sparkNodeTable = SparkNodeTable(nodesMapping, nodesDf) - - val graph = sparkCypherSession.readFrom(sparkNodeTable) - - graph.cypher("MATCH (n) RETURN n").show - } - -} From 3c58c2f05a93de442b528f8b2f96c9457c4f842d Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 19 Feb 2019 16:20:55 +0100 Subject: [PATCH 003/123] Setup TCK testing infrastructure Co-authored-by: Philip Stutz Co-authored-by: Mats Rydberg --- graph/cypher/pom.xml | 25 +- .../graph/cypher/SparkRelationshipTable.scala | 16 + .../src/test/resources/tck/failing_blacklist | 278 ++++++ .../resources/tck/failure_reporting_blacklist | 118 +++ .../src/test/resources/tck/temporal_blacklist | 920 ++++++++++++++++++ .../src/test/resources/tck/wont_fix_blacklist | 193 ++++ .../construction/ScanGraphFactory.scala | 132 +++ .../cypher/tck/SparkCypherTckSuite.scala | 108 ++ 8 files changed, 1775 insertions(+), 15 deletions(-) create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala create mode 100644 graph/cypher/src/test/resources/tck/failing_blacklist create mode 100644 graph/cypher/src/test/resources/tck/failure_reporting_blacklist create mode 100644 graph/cypher/src/test/resources/tck/temporal_blacklist create mode 100644 graph/cypher/src/test/resources/tck/wont_fix_blacklist create mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala create mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 2cf235b42980d..a12394f4dfadc 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -36,11 +36,6 @@ http://spark.apache.org/ - - org.opencypher - okapi-relational - ${okapi.version} - org.apache.spark spark-core_${scala.binary.version} @@ -56,16 +51,16 @@ 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} + + + org.opencypher + okapi-shade + ${okapi.version} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala new file mode 100644 index 0000000000000..2d2f603e5c23d --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala @@ -0,0 +1,16 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.opencypher.okapi.api.io.conversion.RelationshipMapping +import org.opencypher.okapi.relational.api.io.RelationshipTable + +case class SparkRelationshipTable(override val mapping: RelationshipMapping, override val table: DataFrameTable) + extends RelationshipTable(mapping, table) with RecordBehaviour { + + override type Records = SparkRelationshipTable + + override def cache(): SparkRelationshipTable = { + table.cache() + this + } +} 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..0c02e4c472843 --- /dev/null +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -0,0 +1,278 @@ +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 concatenation of lists" +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 "TernaryLogicAcceptanceTest": Scenario "Using null in AND #1" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #2" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #3" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #4" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #5" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #1" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #2" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #3" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #4" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #5" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #1" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #2" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #3" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #4" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #5" +Feature "TernaryLogicAcceptanceTest": Scenario "Using null in IN #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 "OptionalMatchAcceptance": Scenario "Handling optional matches between optionally matched entities" +Feature "OptionalMatchAcceptance": Scenario "Handling optional matches between nulls" +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 "Simple OPTIONAL MATCH on empty graph" +Feature "MatchAcceptance2": Scenario "Handling fixed-length variable length pattern" +Feature "MatchAcceptance2": Scenario "Optionally matching from null nodes should return null" +Feature "MatchAcceptance2": Scenario "OPTIONAL MATCH returns null" +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 "Adding list properties in projection" +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 "MatchAcceptance": Scenario "Accept skip 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 "`properties()` on null" +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 "IN should work with list slices" +Feature "ExpressionAcceptance": Scenario "IN should work with literal list slices" +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 "ColumnNameAcceptance": Scenario "Keeping used expression 4" +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 "AggregationAcceptance": Scenario "`max()` should aggregate strings" +Feature "AggregationAcceptance": Scenario "`min()` should aggregate strings" +Feature "Aggregation": Scenario "`min()` over mixed values" +Feature "Aggregation": Scenario "`min()` over list values" +Feature "Aggregation": Scenario "`max()` over mixed values" +Feature "Aggregation": Scenario "`max()` over strings" +Feature "Aggregation": Scenario "`max()` over mixed numeric values" +Feature "Aggregation": Scenario "`min()` over strings" +Feature "Aggregation": Scenario "`max()` over list values" +Feature "ListOperations": Scenario "Size of list comprehension" +Feature "ListOperations": Scenario "IN should return null when LHS and RHS both ultimately contain null, even if LHS and RHS are of different types (nested list and flat list)" +Feature "ListOperations": Scenario "IN should return false when matching a number with a string - list version" +Feature "ListOperations": Scenario "IN should work with an empty list" +Feature "ListOperations": Scenario "IN should return false when types of LHS and RHS don't match - singleton list" +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 "IN should return null if comparison with null is required for empty list" +Feature "ListOperations": Scenario "IN should work with an empty list in the presence of other list elements: not matching" +Feature "ListOperations": Scenario "IN should work with list slices" +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 "IN should return false for the empty list if the LHS and RHS types differ" +Feature "ListOperations": Scenario "IN should return false when LHS contains a nested list and type mismatch on RHS - singleton version" +Feature "ListOperations": Scenario "Equality between list and literal should return false" +Feature "ListOperations": Scenario "IN should return false when matching a list with a nested list with same elements" +Feature "ListOperations": Scenario "IN should return false when LHS contains a nested list and type mismatch on RHS" +Feature "ListOperations": Scenario "IN should work with an empty list in the presence of other list elements: matching" +Feature "ListOperations": Scenario "IN should return false if no match can be found, despite nulls" +Feature "ListOperations": Scenario "Setting and returning the size of a list property" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected, starting from one extreme" +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 false when types of LHS and RHS don't match - list" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - directed, one way" +Feature "ListOperations": Scenario "Concatenating lists of same type #2" +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 "IN should work with literal list slices" +Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected, starting from two extremes" +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 "IN should return true with previous null match, list version" +Feature "ListOperations": Scenario "Collect and filter using a list comprehension" +Feature "ListOperations": Scenario "Concatenating and returning the size of literal lists" +Feature "ListOperations": Scenario "Concatenating lists of same type #1" \ No newline at end of file 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..31b6dc5fc00b0 --- /dev/null +++ b/graph/cypher/src/test/resources/tck/temporal_blacklist @@ -0,0 +1,920 @@ +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 "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 "TemporalComparisonAcceptance": Scenario "Should compare local date times #1" +Feature "TemporalComparisonAcceptance": Scenario "Should compare local date times #2" \ 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/graph/cypher/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala new file mode 100644 index 0000000000000..14d46cf0d48ff --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala @@ -0,0 +1,132 @@ +package org.apache.spark.graph.cypher.construction + +import java.time.{LocalDate, LocalDateTime} + +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.graph.cypher.conversions.TemporalConversions._ +import org.apache.spark.graph.cypher.conversions.TypeConversions._ +import org.apache.spark.graph.cypher.{SparkCypherSession, SparkNodeTable, SparkRelationshipTable} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, Row} +import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMapping, RelationshipMapping} +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: DataFrame, mapping: EntityMapping): DataFrame = { + + val idCols = mapping.idKeys.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.idKeys.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 = s"source" + val tableEntityEndNodeKey = s"target" + + override def apply(propertyGraph: InMemoryTestGraph) + (implicit sparkCypher: SparkCypherSession): ScanGraph[DataFrameTable] = { + 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 = NodeMapping + .on(tableEntityIdKey) + .withImpliedLabels(labels.toSeq: _*) + .withPropertyKeys(propKeys.keys.toSeq: _*) + + val encodedRecords = encodeIdColumns(records, nodeMapping) + + SparkNodeTable(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 = RelationshipMapping + .on(tableEntityIdKey) + .from(tableEntityStartNodeKey) + .to(tableEntityEndNodeKey) + .relType(relType) + .withPropertyKeys(propKeys.keys.toSeq: _*) + + val encodedRecords = encodeIdColumns(records, relationshipMapping) + + SparkRelationshipTable(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/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala new file mode 100644 index 0000000000000..2327c1067bad7 --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -0,0 +1,108 @@ +package org.apache.spark.graph.cypher.tck + +import java.io.File + +import org.apache.spark.SparkFunSuite +import org.apache.spark.graph.cypher.{SharedCypherContext, SparkCypherSession} +import org.apache.spark.graph.cypher.construction.ScanGraphFactory +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 { + + object TckCapsTag extends 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 wontFixBlacklistFile = getClass.getResource("/tck/wont_fix_blacklist").getFile + private val failureReportingBlacklistFile = getClass.getResource("/tck/failure_reporting_blacklist").getFile + private val scenarios = ScenariosFor(failingBlacklist, temporalBlacklist, wontFixBlacklistFile, failureReportingBlacklistFile) + + forAll(scenarios.whiteList) { scenario => + test(s"[${graphFactory.name}, ${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { + scenario(TCKGraph(graphFactory, sparkCypher.graphs.empty)).execute() + } + } + + forAll(scenarios.blackList) { scenario => + test(s"[${graphFactory.name}, ${BlackList.name}] $scenario", BlackList, TckCapsTag) { + val tckGraph = TCKGraph(graphFactory, sparkCypher.graphs.empty) + + Try(scenario(tckGraph).execute()) match { + case Success(_) => + throw new RuntimeException(s"A blacklisted scenario passed: $scenario") + case Failure(_) => + } + } + } + + test("compute TCK coverage") { + val failingScenarios = Source.fromFile(failingBlacklist).getLines().size + val failingTemporalScenarios = Source.fromFile(temporalBlacklist).getLines().size + val failureReportingScenarios = Source.fromFile(failureReportingBlacklistFile).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, sparkCypher.graphs.empty)).execute()) + } + + ignore("run single scenario") { + scenarios.get("Should add or subtract duration to or from date") + .foreach(scenario => scenario(TCKGraph(graphFactory, sparkCypher.graphs.empty)).execute()) + } +} + From 909154845f1d6dbd3fa7e2ed49e8cf5a1879f349 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 19 Feb 2019 16:32:49 +0100 Subject: [PATCH 004/123] Get all TCK scenarios to pass * except two temporal tests that failed because of loss of precision Co-authored-by: Philip Stutz Co-authored-by: Mats Rydberg --- .../spark/graph/cypher/SparkTable.scala | 258 +++++++++++++++--- .../src/test/resources/tck/temporal_blacklist | 2 + .../construction/ScanGraphFactory.scala | 6 +- 3 files changed, 226 insertions(+), 40 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index 16c0fd10fedb3..bdfd2f5ada8d7 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -3,24 +3,35 @@ package org.apache.spark.graph.cypher import org.apache.spark.graph.cypher.conversions.ExprConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ import org.apache.spark.sql.types.StructField -import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.{Column, DataFrame, RelationalGroupedDataset, functions} +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 -import org.opencypher.okapi.impl.exception.IllegalArgumentException -import org.opencypher.okapi.ir.api.expr.{Aggregator, Expr, Var} +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherValue} +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, NotImplementedException, UnsupportedOperationException} +import org.opencypher.okapi.ir.api.expr._ import org.opencypher.okapi.relational.api.table.Table -import org.opencypher.okapi.relational.impl.planning.{JoinType, Order} +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: DataFrame) extends Table[DataFrameTable] { - override def columnType: Map[String, CypherType] = physicalColumns.map(c => c -> df.cypherTypeForColumn(c)).toMap + 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(cols: String*): DataFrameTable = { if (df.columns.toSeq == cols) { df @@ -29,6 +40,10 @@ object SparkTable { } } + 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 @@ -50,45 +65,214 @@ object SparkTable { 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: Set[(Aggregator, (String, CypherType))]) + (implicit header: RecordHeader, parameters: CypherMap): DataFrameTable = { + + def withInnerExpr(expr: Expr)(f: Column => Column) = + f(expr.asSparkSQLExpr(header, df, parameters)) + + val data: Either[RelationalGroupedDataset, DataFrame] = + 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 (aggFunc, (columnName, cypherType)) => + aggFunc match { + case Avg(expr) => + withInnerExpr(expr)( + functions + .avg(_) + .cast(cypherType.getSparkType) + .as(columnName)) + + case CountStar(_) => + functions.count(functions.lit(0)).as(columnName) + + // TODO: Consider not implicitly projecting the aggFunc expr here, but rewriting it into a variable in logical planning or IR construction + case Count(expr, distinct) => withInnerExpr(expr) { column => + val count = { + if (distinct) functions.countDistinct(column) + else functions.count(column) + } + count.as(columnName) + } + + case Max(expr) => + withInnerExpr(expr)(functions.max(_).as(columnName)) + + case Min(expr) => + withInnerExpr(expr)(functions.min(_).as(columnName)) + + case Sum(expr) => + withInnerExpr(expr)(functions.sum(_).as(columnName)) + + case Collect(expr, distinct) => withInnerExpr(expr) { column => + val list = { + if (distinct) functions.collect_set(column) + else functions.collect_list(column) + } + // sort for deterministic aggregation results + val sorted = functions.sort_array(list) + sorted.as(columnName) + } + + case x => + throw NotImplementedException(s"Aggregation function $x") + } + } + + 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 _ => + + // TODO: the join produced corrupt data when the previous operator was a cross. We work around that by using a + // subsequent select. This can be removed, once https://issues.apache.org/jira/browse/SPARK-23855 is solved or we + // upgrade to Spark 2.3.0 + val potentiallyCorruptedResult = df.safeJoin(other.df, joinCols, joinTypeString) + potentiallyCorruptedResult.select("*") + } + } + + override def distinct: DataFrameTable = distinct(df.columns: _*) + + // workaround for https://issues.apache.org/jira/browse/SPARK-26572 + override def distinct(colNames: String*): DataFrameTable = { + val uniqueSuffix = "_temp_distinct" + + val originalColNames = df.columns + + val renamings = originalColNames.map { c => + if (colNames.contains(c)) c -> s"$c$uniqueSuffix" + else c -> c + }.toMap + + val renamedDf = df.safeRenameColumns(colNames.map(c => c -> renamings(c)).toMap) + + val extractRowFromGrouping = originalColNames.map(c => functions.first(renamings(c)) as c) + val groupedDf = renamedDf + .groupBy(colNames.map(c => functions.col(renamings(c))): _*) + .agg(extractRowFromGrouping.head, extractRowFromGrouping.tail: _*) + + groupedDf.safeDropColumns(colNames.map(renamings): _*) + } + override def withColumnsRenamed(columnRenamings: Map[String, String]): DataFrameTable = { df.safeRenameColumns(columnRenamings) } - override def filter(expr: Expr)(implicit header: RecordHeader, parameters: CypherValue.CypherMap): DataFrameTable = ??? - override def drop(cols: String*): DataFrameTable = ??? - override def join( - other: DataFrameTable, - joinType: JoinType, - joinCols: (String, String)* - ): DataFrameTable = ??? - override def unionAll(other: DataFrameTable): DataFrameTable = ??? - override def orderBy(sortItems: (Expr, Order)*) - ( - implicit header: RecordHeader, - parameters: CypherValue.CypherMap - ): DataFrameTable = ??? - override def skip(n: Long): DataFrameTable = ??? - override def limit(n: Long): DataFrameTable = ??? - override def distinct: DataFrameTable = ??? - override def group( - by: Set[Var], - aggregations: Set[(Aggregator, (String, CypherType))] - ) - ( - implicit header: RecordHeader, - parameters: CypherValue.CypherMap - ): DataFrameTable = ??? - - - override def show(rows: Int): Unit = ??? - - override def columnsFor(returnItem: String): Set[String] = ??? - override def rows: Iterator[String => CypherValue.CypherValue] = ??? - override def size: Long = ??? + 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) + + override def columnsFor(returnItem: String): Set[String] = + throw UnsupportedOperationException("A DataFrameTable does not have return items") } implicit class DataFrameOps(val df: DataFrame) extends AnyVal { + def safeJoin(other: DataFrame, joinCols: Seq[(String, String)], joinType: String): DataFrame = { + require(joinCols.map(_._1).forall(col => !other.columns.contains(col))) + require(joinCols.map(_._2).forall(col => !df.columns.contains(col))) + + val joinExpr = joinCols.map { + case (l, r) => df.col(l) === other.col(r) + }.reduce((acc, expr) => acc && expr) + + df.join(other, joinExpr, joinType) + } + + def safeDropColumns(names: String*): DataFrame = { + 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)) diff --git a/graph/cypher/src/test/resources/tck/temporal_blacklist b/graph/cypher/src/test/resources/tck/temporal_blacklist index 31b6dc5fc00b0..72dd5d62f51e7 100644 --- a/graph/cypher/src/test/resources/tck/temporal_blacklist +++ b/graph/cypher/src/test/resources/tck/temporal_blacklist @@ -360,7 +360,9 @@ 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 #1" Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #2" +Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #3" Feature "TemporalCreateAcceptance": Scenario "Should store local time #1" Feature "TemporalCreateAcceptance": Scenario "Should store local time #2" Feature "TemporalCreateAcceptance": Scenario "Should store local time #3" diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala index 14d46cf0d48ff..28e862a149735 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala @@ -46,9 +46,9 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { apply(CreateGraphFactory(createQuery)) } - val tableEntityIdKey = "id" - val tableEntityStartNodeKey = s"source" - val tableEntityEndNodeKey = s"target" + val tableEntityIdKey = "___id" + val tableEntityStartNodeKey = "___source" + val tableEntityEndNodeKey = "___target" override def apply(propertyGraph: InMemoryTestGraph) (implicit sparkCypher: SparkCypherSession): ScanGraph[DataFrameTable] = { From 7abaddf3727b9325c18a65e47bc782a11999e166 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 19 Feb 2019 17:49:37 +0100 Subject: [PATCH 005/123] Remove unnecessary cast --- .../spark/graph/cypher/conversions/CypherValueEncoders.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala index fd0502a1a6fe1..67659b3db16a5 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala @@ -23,7 +23,7 @@ object CypherValueEncoders { } implicit def cypherNodeEncoder: ExpressionEncoder[SparkCypherNode] = { - kryo[SparkCypherNode].asInstanceOf[ExpressionEncoder[SparkCypherNode]] + kryo[SparkCypherNode] } implicit def cypherRelationshipEncoder: ExpressionEncoder[SparkCypherRelationship] = { From 3f0acbf53f2cda6063377db822db81dc29c688a8 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 20 Feb 2019 10:56:18 +0100 Subject: [PATCH 006/123] Remove unnecessary Neo4j specific dependencies Co-authored-by: Philip Stutz --- graph/cypher/pom.xml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index a12394f4dfadc..2f5377177018c 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -91,6 +91,15 @@ ${okapi.version} test + + + + org.antlr + antlr4-runtime + 4.7.1 + test + + From 9541508fe4bb6b6db62299390d399e5c64b064f8 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 20 Feb 2019 14:09:26 +0100 Subject: [PATCH 007/123] Exclude Antlr from spark-sql deps * workaround to get TCK to work proper Co-authored-by: Philip Stutz --- graph/cypher/pom.xml | 29 ++++++++++++++++------------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 2f5377177018c..ee3b4ad3ff146 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -45,11 +45,13 @@ org.apache.spark spark-sql_${scala.binary.version} ${project.version} - - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} + + + + org.antlr + antlr4-runtime + + org.apache.spark @@ -77,14 +79,15 @@ ${project.version} test-jar test + + + + org.antlr + antlr4-runtime + + - - org.apache.spark - spark-catalyst_${scala.binary.version} - ${project.version} - test-jar - test - + org.opencypher okapi-tck @@ -92,7 +95,7 @@ test - + org.antlr antlr4-runtime From 14bdada263b483ae26559895be043931779d98fa Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 20 Feb 2019 15:43:00 +0100 Subject: [PATCH 008/123] Add spark-graph-api Removed cherry-pick conflict in pom.xml Co-authored-by: Martin Junghanns --- .../spark/graph/api/v1/CypherEngine.scala | 10 +++++++ .../spark/graph/api/v1/CypherResult.scala | 7 +++++ .../graph/api/v1/GraphElementDataFrame.scala | 29 +++++++++++++++++++ .../spark/graph/api/v1/PropertyGraph.scala | 15 ++++++++++ 4 files changed, 61 insertions(+) create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala new file mode 100644 index 0000000000000..6a934ab3e57d6 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala @@ -0,0 +1,10 @@ +package org.apache.spark.graph.api.v1 + +trait CypherEngine { + def cypher(graph: PropertyGraph, query: String): CypherResult + + def createGraph( + nodes: Seq[NodeDataFrame], + relationships: Seq[RelationshipDataFrame] + ): PropertyGraph +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala new file mode 100644 index 0000000000000..cbc43332d0922 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala @@ -0,0 +1,7 @@ +package org.apache.spark.graph.api.v1 + +import org.apache.spark.sql.DataFrame + +sealed trait CypherResult { + def df: DataFrame +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala new file mode 100644 index 0000000000000..5d80f08b7f9d6 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala @@ -0,0 +1,29 @@ +package org.apache.spark.graph.api.v1 + +import org.apache.spark.sql.DataFrame + +trait GraphElementDataFrame { + + def df: DataFrame + + def idColumn: String + + def properties: Map[String, String] +} + +case class NodeDataFrame( + df: DataFrame, + idColumn: String, + properties: Map[String, String], + labels: Set[String], + optionalLabels: Map[String, String] +) extends GraphElementDataFrame + +case class RelationshipDataFrame( + df: DataFrame, + idColumn: String, + properties: Map[String, String], + relationshipType: String, + sourceIdColumn: String, + targetIdColumn: String +) extends GraphElementDataFrame diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala new file mode 100644 index 0000000000000..823ebb9eb2d18 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala @@ -0,0 +1,15 @@ +package org.apache.spark.graph.api.v1 + +import org.apache.spark.sql.DataFrame + +trait PropertyGraph { + def cypherEngine: CypherEngine + + def cypher(query: String): CypherResult = + cypherEngine.cypher(this, query) + + def nodes: DataFrame + def vertices: DataFrame = nodes + def relationships: DataFrame + def edges: DataFrame = relationships +} From bfed6b548c5aa6bc4863cef8ff45a0d5c7f0929f Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 20 Feb 2019 17:41:21 +0100 Subject: [PATCH 009/123] Add adapters for spark-graph-cypher and example Co-authored-by: Philip Stutz --- .../graph/api/{v1 => }/CypherEngine.scala | 4 +- .../apache/spark/graph/api/CypherResult.scala | 7 ++++ .../api/{v1 => }/GraphElementDataFrame.scala | 14 +++---- .../graph/api/{v1 => }/PropertyGraph.scala | 2 +- .../spark/graph/api/v1/CypherResult.scala | 7 ---- graph/cypher/pom.xml | 20 ++++----- .../graph/cypher/SparkCypherSession.scala | 41 +++++++++++++++++-- .../cypher/adapters/MappingAdapter.scala | 18 ++++++++ .../adapters/RelationalGraphAdapter.scala | 15 +++++++ .../spark/graph/cypher/BasicMatchSuite.scala | 23 ++++++++--- .../graph/cypher/SharedCypherContext.scala | 8 ++-- .../cypher/tck/SparkCypherTckSuite.scala | 8 ++-- 12 files changed, 123 insertions(+), 44 deletions(-) rename graph/api/src/main/scala/org/apache/spark/graph/api/{v1 => }/CypherEngine.scala (62%) create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/CypherResult.scala rename graph/api/src/main/scala/org/apache/spark/graph/api/{v1 => }/GraphElementDataFrame.scala (63%) rename graph/api/src/main/scala/org/apache/spark/graph/api/{v1 => }/PropertyGraph.scala (88%) delete mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala similarity index 62% rename from graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala index 6a934ab3e57d6..7e9af74f5e111 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherEngine.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala @@ -1,10 +1,10 @@ -package org.apache.spark.graph.api.v1 +package org.apache.spark.graph.api trait CypherEngine { def cypher(graph: PropertyGraph, query: String): CypherResult def createGraph( nodes: Seq[NodeDataFrame], - relationships: Seq[RelationshipDataFrame] + relationships: Seq[RelationshipDataFrame] = Seq.empty ): PropertyGraph } 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..998c795706ab4 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherResult.scala @@ -0,0 +1,7 @@ +package org.apache.spark.graph.api + +import org.apache.spark.sql.DataFrame + +trait CypherResult { + def df: DataFrame +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala similarity index 63% rename from graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala index 5d80f08b7f9d6..ae06ee8badbae 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/GraphElementDataFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.api.v1 +package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame @@ -14,16 +14,16 @@ trait GraphElementDataFrame { case class NodeDataFrame( df: DataFrame, idColumn: String, - properties: Map[String, String], - labels: Set[String], - optionalLabels: Map[String, String] + labels: Set[String] = Set.empty, + properties: Map[String, String] = Map.empty, + optionalLabels: Map[String, String] = Map.empty ) extends GraphElementDataFrame case class RelationshipDataFrame( df: DataFrame, idColumn: String, - properties: Map[String, String], - relationshipType: String, sourceIdColumn: String, - targetIdColumn: String + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String] = Map.empty ) extends GraphElementDataFrame diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraph.scala similarity index 88% rename from graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraph.scala index 823ebb9eb2d18..2b4f3b7a12278 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/PropertyGraph.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraph.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.api.v1 +package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala deleted file mode 100644 index cbc43332d0922..0000000000000 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/v1/CypherResult.scala +++ /dev/null @@ -1,7 +0,0 @@ -package org.apache.spark.graph.api.v1 - -import org.apache.spark.sql.DataFrame - -sealed trait CypherResult { - def df: DataFrame -} diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index ee3b4ad3ff146..4cb700c21af2a 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -53,16 +53,16 @@ - - 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} + + + org.opencypher + okapi-shade + ${okapi.version} + diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index cc616806bfb27..9f767bf6481fd 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -1,16 +1,20 @@ package org.apache.spark.graph.cypher +import org.apache.spark.graph.api._ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.sql.SparkSession +import org.apache.spark.graph.cypher.adapters.MappingAdapter._ +import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.opencypher.okapi.impl.exception.IllegalArgumentException 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.RelationalEntityTableFactory +import org.opencypher.okapi.relational.api.table.{RelationalCypherRecords, RelationalEntityTableFactory} object SparkCypherSession { def create(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) } -class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] { +class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherEngine { override type Result = RelationalCypherResult[DataFrameTable] override type Records = SparkCypherRecords @@ -25,6 +29,37 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe } override def entityTables: RelationalEntityTableFactory[DataFrameTable] = ??? + + override def createGraph( + nodes: Seq[NodeDataFrame], + relationships: Seq[RelationshipDataFrame] + ): PropertyGraph = { + require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") + val nodeTables = nodes.map { nodeDataFrame => SparkNodeTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } + val relTables = relationships.map { relDataFrame => SparkRelationshipTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } + + RelationalGraphAdapter(this, graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*)) + } + + override def cypher(graph: PropertyGraph, query: String): CypherResult = { + val relationalGraph = graph match { + case RelationalGraphAdapter(_, relGraph) => relGraph + case other => throw IllegalArgumentException( + expected = "A graph that has been created by `SparkCypherSession.createGraph`", + actual = other.getClass.getSimpleName + ) + } + + val result: DataFrame = relationalGraph.cypher(query).records match { + case relationalRecords: RelationalCypherRecords[_] => relationalRecords.table match { + case table: DataFrameTable => table.df + } + } + + new CypherResult { + override def df: DataFrame = result + } + } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala new file mode 100644 index 0000000000000..57dcf42617e7c --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala @@ -0,0 +1,18 @@ +package org.apache.spark.graph.cypher.adapters + +import org.apache.spark.graph.api.{NodeDataFrame, RelationshipDataFrame} +import org.opencypher.okapi.api.io.conversion.{NodeMapping, RelationshipMapping} + +object MappingAdapter { + + implicit class RichNodeDataFrame(val nodeDf: NodeDataFrame) extends AnyVal { + def toNodeMapping: NodeMapping = + NodeMapping(nodeDf.idColumn, nodeDf.labels, nodeDf.optionalLabels, nodeDf.properties) + + } + + implicit class RichRelationshipDataFrame(val relDf: RelationshipDataFrame) extends AnyVal { + def toRelationshipMapping: RelationshipMapping = + RelationshipMapping(relDf.idColumn, relDf.sourceIdColumn, relDf.targetIdColumn, Left(relDf.relationshipType), relDf.properties) + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala new file mode 100644 index 0000000000000..601858b112c1b --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala @@ -0,0 +1,15 @@ +package org.apache.spark.graph.cypher.adapters + +import org.apache.spark.graph.api.{CypherEngine, PropertyGraph} +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.sql.DataFrame +import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph + +case class RelationalGraphAdapter( + cypherEngine: CypherEngine, + graph: RelationalCypherGraph[DataFrameTable]) extends PropertyGraph { + + override def nodes: DataFrame = graph.nodes("n").table.df + + override def relationships: DataFrame = graph.relationships("r").table.df +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index bc20e05068eb9..86a4c97193c51 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -1,19 +1,30 @@ package org.apache.spark.graph.cypher import org.apache.spark.SparkFunSuite +import org.apache.spark.graph.api.{NodeDataFrame, PropertyGraph} +import org.apache.spark.sql.DataFrame +import org.opencypher.okapi.api.graph.{PropertyGraph => OkapiPropertyGraph} import org.opencypher.okapi.api.io.conversion.NodeMapping class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { - test("match single node pattern") { - val nodesDf = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") - val nodesMapping = NodeMapping.on("id").withImpliedLabel("Person").withPropertyKey("name") + test("match single node pattern using spark-graph-cypher") { + val nodeData: DataFrame = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val nodesMapping: NodeMapping = NodeMapping.on("id").withImpliedLabel("Person").withPropertyKey("name") + val sparkNodeTable: SparkNodeTable = SparkNodeTable(nodesMapping, nodeData) - val sparkNodeTable = SparkNodeTable(nodesMapping, nodesDf) - - val graph = sparkCypher.readFrom(sparkNodeTable) + val graph: OkapiPropertyGraph = cypherEngine.readFrom(sparkNodeTable) graph.cypher("MATCH (n) RETURN n").show } + test("match single node pattern using spark-graph-api") { + val nodeData = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val nodeDataFrame: NodeDataFrame = NodeDataFrame(nodeData, "id", Set("Person"), Map("name" -> "name")) + + val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) + + graph.cypher("MATCH (n) RETURN n").df.show() + } + } diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala index 5ba8b9ca3d67e..22f2e9abfe709 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala @@ -4,17 +4,17 @@ import org.apache.spark.sql.test.SharedSQLContext trait SharedCypherContext extends SharedSQLContext { - private var _sparkCypher: SparkCypherSession = _ + private var _cypherEngine: SparkCypherSession = _ - protected implicit def sparkCypher: SparkCypherSession = _sparkCypher + protected implicit def cypherEngine: SparkCypherSession = _cypherEngine override def beforeAll() { super.beforeAll() - _sparkCypher = SparkCypherSession.create + _cypherEngine = SparkCypherSession.create } protected override def afterAll(): Unit = { - _sparkCypher = null + _cypherEngine = null super.afterAll() } } diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala index 2327c1067bad7..3a981a06cd5d3 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -29,13 +29,13 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { forAll(scenarios.whiteList) { scenario => test(s"[${graphFactory.name}, ${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { - scenario(TCKGraph(graphFactory, sparkCypher.graphs.empty)).execute() + scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute() } } forAll(scenarios.blackList) { scenario => test(s"[${graphFactory.name}, ${BlackList.name}] $scenario", BlackList, TckCapsTag) { - val tckGraph = TCKGraph(graphFactory, sparkCypher.graphs.empty) + val tckGraph = TCKGraph(graphFactory, cypherEngine.graphs.empty) Try(scenario(tckGraph).execute()) match { case Success(_) => @@ -97,12 +97,12 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { CypherTCK .parseFilesystemFeature(file) .scenarios - .foreach(scenario => scenario(TCKGraph(graphFactory, sparkCypher.graphs.empty)).execute()) + .foreach(scenario => scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute()) } ignore("run single scenario") { scenarios.get("Should add or subtract duration to or from date") - .foreach(scenario => scenario(TCKGraph(graphFactory, sparkCypher.graphs.empty)).execute()) + .foreach(scenario => scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute()) } } From 3f54f6ccb423661e0a27c0fadc86d514932047ac Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 20 Feb 2019 18:02:34 +0100 Subject: [PATCH 010/123] Use named arguments in example for better showcasing Co-authored-by: Philip Stutz --- .../scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 86a4c97193c51..63e2ccf447791 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -19,8 +19,8 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { } test("match single node pattern using spark-graph-api") { - val nodeData = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") - val nodeDataFrame: NodeDataFrame = NodeDataFrame(nodeData, "id", Set("Person"), Map("name" -> "name")) + val nodeData: DataFrame = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) From 852e9908c53a6e93356181b3475089be5418ff32 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 22 Feb 2019 12:07:52 +0100 Subject: [PATCH 011/123] Adapt to latest OKAPI changes Co-authored-by: Martin Junghanns --- .../graph/cypher/SparkCypherSession.scala | 10 ++------- .../spark/graph/cypher/SparkTable.scala | 13 ++++++------ .../spark/graph/cypher/BasicMatchSuite.scala | 21 +++++++++---------- 3 files changed, 18 insertions(+), 26 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 9f767bf6481fd..1e6364ea4992a 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.{DataFrame, SparkSession} import org.opencypher.okapi.impl.exception.IllegalArgumentException 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.{RelationalCypherRecords, RelationalEntityTableFactory} +import org.opencypher.okapi.relational.api.table.RelationalEntityTableFactory object SparkCypherSession { def create(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) @@ -50,14 +50,8 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe ) } - val result: DataFrame = relationalGraph.cypher(query).records match { - case relationalRecords: RelationalCypherRecords[_] => relationalRecords.table match { - case table: DataFrameTable => table.df - } - } - new CypherResult { - override def df: DataFrame = result + override def df: DataFrame = relationalGraph.cypher(query).records.table.df } } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index bdfd2f5ada8d7..b557d7eaf1e11 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -8,7 +8,7 @@ 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, NotImplementedException, UnsupportedOperationException} +import org.opencypher.okapi.impl.exception.{IllegalArgumentException, NotImplementedException} import org.opencypher.okapi.ir.api.expr._ import org.opencypher.okapi.relational.api.table.Table import org.opencypher.okapi.relational.impl.planning._ @@ -32,11 +32,13 @@ object SparkTable { override def size: Long = df.count() - override def select(cols: String*): DataFrameTable = { - if (df.columns.toSeq == cols) { + 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 { - df.select(cols.map(df.col): _*) + // 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) }: _*) } } @@ -248,9 +250,6 @@ object SparkTable { def unpersist(): DataFrameTable = df.unpersist() def unpersist(blocking: Boolean): DataFrameTable = df.unpersist(blocking) - - override def columnsFor(returnItem: String): Set[String] = - throw UnsupportedOperationException("A DataFrameTable does not have return items") } implicit class DataFrameOps(val df: DataFrame) extends AnyVal { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 63e2ccf447791..14dc7c3632f43 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -1,30 +1,29 @@ package org.apache.spark.graph.cypher import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.api.{NodeDataFrame, PropertyGraph} +import org.apache.spark.graph.api.{NodeDataFrame, PropertyGraph, RelationshipDataFrame} import org.apache.spark.sql.DataFrame -import org.opencypher.okapi.api.graph.{PropertyGraph => OkapiPropertyGraph} -import org.opencypher.okapi.api.io.conversion.NodeMapping class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { - test("match single node pattern using spark-graph-cypher") { + test("match single node pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") - val nodesMapping: NodeMapping = NodeMapping.on("id").withImpliedLabel("Person").withPropertyKey("name") - val sparkNodeTable: SparkNodeTable = SparkNodeTable(nodesMapping, nodeData) + val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) - val graph: OkapiPropertyGraph = cypherEngine.readFrom(sparkNodeTable) + val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) - graph.cypher("MATCH (n) RETURN n").show + graph.cypher("MATCH (n) RETURN n").df.show() } - test("match single node pattern using spark-graph-api") { + test("match simple pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(Array[Byte](0), Array[Byte](0), Array[Byte](1)))).toDF("id", "source", "target") val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) + val relationshipFrame: RelationshipDataFrame = RelationshipDataFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) + val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) - graph.cypher("MATCH (n) RETURN n").df.show() + graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } } From fdef716a9159bdab7668c611b9d3ecdc1cd3aecd Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 22 Feb 2019 18:51:25 +0100 Subject: [PATCH 012/123] Support for turning Cypher results into nodes/rels for property graph Co-authored-by: Martin Junghanns --- .../apache/spark/graph/api/CypherResult.scala | 4 + .../graph/api/GraphElementDataFrame.scala | 3 +- .../graph/cypher/SparkCypherResult.scala | 86 +++++++++++++++++++ .../graph/cypher/SparkCypherSession.scala | 4 +- .../cypher/adapters/MappingAdapter.scala | 2 +- .../spark/graph/cypher/BasicMatchSuite.scala | 38 +++++++- 6 files changed, 127 insertions(+), 10 deletions(-) create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala 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 index 998c795706ab4..26e6e4f2b7f23 100644 --- 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 @@ -4,4 +4,8 @@ import org.apache.spark.sql.DataFrame trait CypherResult { def df: DataFrame + + def nodeDataFrame(varName: String): Seq[NodeDataFrame] + + def relationshipDataFrame(varName: String): Seq[RelationshipDataFrame] } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala index ae06ee8badbae..510ad44e33293 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala @@ -15,8 +15,7 @@ case class NodeDataFrame( df: DataFrame, idColumn: String, labels: Set[String] = Set.empty, - properties: Map[String, String] = Map.empty, - optionalLabels: Map[String, String] = Map.empty + properties: Map[String, String] = Map.empty ) extends GraphElementDataFrame case class RelationshipDataFrame( diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala new file mode 100644 index 0000000000000..55fec43434ec0 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -0,0 +1,86 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.api.{CypherResult, NodeDataFrame, RelationshipDataFrame} +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.sql.{DataFrame, functions} +import org.opencypher.okapi.api.schema.Schema +import org.opencypher.okapi.api.types.{CTNode, CTRelationship} +import org.opencypher.okapi.impl.exception.IllegalArgumentException +import org.opencypher.okapi.ir.api.expr._ +import org.opencypher.okapi.relational.api.table.RelationalCypherRecords +import org.opencypher.okapi.relational.impl.table.RecordHeader + +case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable], schema: Schema) extends CypherResult { + + override val df: DataFrame = relationalTable.table.df + + private val header: RecordHeader = relationalTable.header + + // TODO: Error handling + // TODO: Distinct if more than one entityVar + override def nodeDataFrame(varName: String): Seq[NodeDataFrame] = { + val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) + + val idColumn = header.column(nodeVar) + val possibleLabels = header.labelsFor(nodeVar).map(_.label.name) + val labelCombinations = schema.combinationsFor(possibleLabels).toSeq + + val labelToColumns = header.labelsFor(nodeVar).map(expr => expr.label.name -> header.column(expr)).toMap + val propertyToColumns = header.propertiesFor(nodeVar).map(expr => expr.key.name -> header.column(expr)).toMap + + val allLabelColumns = labelToColumns.values.toSet + + labelCombinations.map { labels => + val trueLabels = labels.map(labelToColumns) + val falseLabels = allLabelColumns -- trueLabels + + val propertyKeys = schema.nodePropertyKeys(labels).keySet + val properties = propertyToColumns.filter { case (propertyKey, _) => propertyKeys.contains(propertyKey) } + val propertyColumns = properties.values.toSeq + + val selectColumns = (idColumn +: propertyColumns).map(col => s"`$col`") + val labelCombinationDf = df + .filter(trueLabels.map(df.col).map(_ === true).reduce(_ && _)) + .filter(falseLabels.map(df.col).map(_ === false).foldLeft(functions.lit(true))(_ && _)) + .select(selectColumns.head, selectColumns.tail: _*) + + NodeDataFrame(labelCombinationDf, idColumn, labels, properties) + } + } + + // TODO: Error handling + // TODO: Distinct if more than one entityVar + override def relationshipDataFrame(varName: String): Seq[RelationshipDataFrame] = { + val relVar: RelationshipVar = find(RelationshipVar(varName)(CTRelationship)) + + val idColumn = header.column(relVar) + val relTypes = header.typesFor(relVar).map(_.relType.name).toSeq + val sourceIdColumn = header.column(header.startNodeFor(relVar)) + val targetIdColumn = header.column(header.endNodeFor(relVar)) + + val relTypeToColumns = header.typesFor(relVar).map(expr => expr.relType.name -> header.column(expr)).toMap + val propertyToColumns = header.propertiesFor(relVar).map(expr => expr.key.name -> header.column(expr)).toMap + + relTypes.map { relType => + val trueRelType = relTypeToColumns(relType) + val propertyKeys = schema.relationshipPropertyKeys(relType).keySet + val properties = propertyToColumns.filter { case (propertyKey, _) => propertyKeys.contains(propertyKey) } + val propertyColumns = properties.values.toSeq + + val selectColumns = (idColumn +: sourceIdColumn +: targetIdColumn +: propertyColumns).map(col => s"`$col`") + val relTypeDf = df + .filter(df.col(trueRelType) === true) + .select(selectColumns.head, selectColumns.tail: _*) + + RelationshipDataFrame(relTypeDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + } + } + + // TODO use header.entityVars ? + private def find[T <: ReturnItem](lookup: Var): T = relationalTable.header.idExpressions(lookup) + .collectFirst { case expr if expr.withoutType == lookup.withoutType => expr } + .getOrElse(throw IllegalArgumentException( + expected = s"One of ${relationalTable.header.vars.map(_.withoutType).toList}", + actual = lookup.name + )).asInstanceOf[T] +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 1e6364ea4992a..d510dc5bda383 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -50,9 +50,7 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe ) } - new CypherResult { - override def df: DataFrame = relationalGraph.cypher(query).records.table.df - } + SparkCypherResult(relationalGraph.cypher(query).records, relationalGraph.schema) } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala index 57dcf42617e7c..83c19057d38b8 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala @@ -7,7 +7,7 @@ object MappingAdapter { implicit class RichNodeDataFrame(val nodeDf: NodeDataFrame) extends AnyVal { def toNodeMapping: NodeMapping = - NodeMapping(nodeDf.idColumn, nodeDf.labels, nodeDf.optionalLabels, nodeDf.properties) + NodeMapping(nodeDf.idColumn, nodeDf.labels, propertyMapping = nodeDf.properties) } diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 14dc7c3632f43..e9d55c9ba7214 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -1,13 +1,13 @@ package org.apache.spark.graph.cypher import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.api.{NodeDataFrame, PropertyGraph, RelationshipDataFrame} +import org.apache.spark.graph.api.{CypherResult, NodeDataFrame, PropertyGraph, RelationshipDataFrame} import org.apache.spark.sql.DataFrame class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("match single node pattern using spark-graph-api") { - val nodeData: DataFrame = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") + val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) @@ -16,8 +16,8 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { } test("match simple pattern using spark-graph-api") { - val nodeData: DataFrame = spark.createDataFrame(Seq(Array[Byte](0) -> "Alice", Array[Byte](1) -> "Bob")).toDF("id", "name") - val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(Array[Byte](0), Array[Byte](0), Array[Byte](1)))).toDF("id", "source", "target") + val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") + val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) val relationshipFrame: RelationshipDataFrame = RelationshipDataFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") @@ -26,4 +26,34 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } + // TODO: requires fixing escaping in CAPS + ignore("create property graph from query results") { + val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), "Alice", 42), Tuple3(id(1), "Bob", 23), Tuple3(id(2), "Eve", 19))).toDF("id", "name", "age") + val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") + val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") + val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(2), id(0), id(2)), Tuple3(id(3), id(1), id(3)), Tuple3(id(4), id(2), id(2)))).toDF("id", "source", "target") + val personDataFrame: NodeDataFrame = NodeDataFrame(df = personData, idColumn = "id", labels = Set("Student"), properties = Map("name" -> "name", "age" -> "age")) + val universityDataFrame: NodeDataFrame = NodeDataFrame(df = universityData, idColumn = "id", labels = Set("University"), properties = Map("title" -> "title")) + val knowsDataFrame: RelationshipDataFrame = RelationshipDataFrame(df = knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val studyAtDataFrame: RelationshipDataFrame = RelationshipDataFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") + + val graph: PropertyGraph = cypherEngine.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) + val result: CypherResult = graph.cypher( + """ + |MATCH (p:Student)-[:STUDY_AT]->(u:University), + | (p)-[k:KNOWS]->(o:Student)-[:STUDY_AT]->(u) + |WHERE u.title = 'UC Berkeley' + |RETURN p, o, k + |""".stripMargin) + + val berkeleyStudents: Seq[NodeDataFrame] = result.nodeDataFrame("p") + val berkeleyStudentFriends: Seq[NodeDataFrame] = result.nodeDataFrame("o") + val knows: Seq[RelationshipDataFrame] = result.relationshipDataFrame("k") + + val berkeleyGraph: PropertyGraph = cypherEngine.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) + berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() + + } + + private def id(l: Long): Array[Byte] = BigInt(l).toByteArray } From 542a3e7a0502dcd5aae79a5651767481dd03f2ce Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 25 Feb 2019 11:17:47 +0000 Subject: [PATCH 013/123] Fix round trip example by replacing dots when importing Co-authored-by: Martin Junghanns --- .../apache/spark/graph/api/CypherResult.scala | 4 +-- .../graph/cypher/SparkCypherResult.scala | 27 ++++++++++++------- .../graph/cypher/SparkCypherSession.scala | 6 ++--- .../spark/graph/cypher/SparkNodeTable.scala | 24 ++++++++++++++++- .../graph/cypher/SparkRelationshipTable.scala | 25 ++++++++++++++++- .../spark/graph/cypher/SparkTable.scala | 9 ++++--- .../spark/graph/cypher/BasicMatchSuite.scala | 9 +++---- .../construction/ScanGraphFactory.scala | 4 +-- 8 files changed, 80 insertions(+), 28 deletions(-) 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 index 26e6e4f2b7f23..d690e499eca04 100644 --- 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 @@ -5,7 +5,7 @@ import org.apache.spark.sql.DataFrame trait CypherResult { def df: DataFrame - def nodeDataFrame(varName: String): Seq[NodeDataFrame] + def nodeDataFrames(varName: String): Seq[NodeDataFrame] - def relationshipDataFrame(varName: String): Seq[RelationshipDataFrame] + def relationshipDataFrames(varName: String): Seq[RelationshipDataFrame] } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index 55fec43434ec0..66c3adf89ce84 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -10,15 +10,19 @@ import org.opencypher.okapi.ir.api.expr._ import org.opencypher.okapi.relational.api.table.RelationalCypherRecords import org.opencypher.okapi.relational.impl.table.RecordHeader -case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable], schema: Schema) extends CypherResult { +case class SparkCypherResult( + relationalTable: RelationalCypherRecords[DataFrameTable], + schema: Schema +) extends CypherResult { override val df: DataFrame = relationalTable.table.df private val header: RecordHeader = relationalTable.header + private lazy val hasMultipleEntities = header.entityVars.size > 1 + // TODO: Error handling - // TODO: Distinct if more than one entityVar - override def nodeDataFrame(varName: String): Seq[NodeDataFrame] = { + override def nodeDataFrames(varName: String): Seq[NodeDataFrame] = { val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) val idColumn = header.column(nodeVar) @@ -38,19 +42,20 @@ case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameT val properties = propertyToColumns.filter { case (propertyKey, _) => propertyKeys.contains(propertyKey) } val propertyColumns = properties.values.toSeq - val selectColumns = (idColumn +: propertyColumns).map(col => s"`$col`") + val selectColumns = (idColumn +: propertyColumns).map(escape) val labelCombinationDf = df .filter(trueLabels.map(df.col).map(_ === true).reduce(_ && _)) .filter(falseLabels.map(df.col).map(_ === false).foldLeft(functions.lit(true))(_ && _)) .select(selectColumns.head, selectColumns.tail: _*) - NodeDataFrame(labelCombinationDf, idColumn, labels, properties) + val distinctDf = if (hasMultipleEntities) labelCombinationDf.dropDuplicates(idColumn) else labelCombinationDf + + NodeDataFrame(distinctDf, idColumn, labels, properties) } } // TODO: Error handling - // TODO: Distinct if more than one entityVar - override def relationshipDataFrame(varName: String): Seq[RelationshipDataFrame] = { + override def relationshipDataFrames(varName: String): Seq[RelationshipDataFrame] = { val relVar: RelationshipVar = find(RelationshipVar(varName)(CTRelationship)) val idColumn = header.column(relVar) @@ -67,15 +72,19 @@ case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameT val properties = propertyToColumns.filter { case (propertyKey, _) => propertyKeys.contains(propertyKey) } val propertyColumns = properties.values.toSeq - val selectColumns = (idColumn +: sourceIdColumn +: targetIdColumn +: propertyColumns).map(col => s"`$col`") + val selectColumns = (idColumn +: sourceIdColumn +: targetIdColumn +: propertyColumns).map(escape) val relTypeDf = df .filter(df.col(trueRelType) === true) .select(selectColumns.head, selectColumns.tail: _*) - RelationshipDataFrame(relTypeDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + val distinctDf = if (hasMultipleEntities) relTypeDf.dropDuplicates(idColumn) else relTypeDf + + RelationshipDataFrame(distinctDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) } } + private def escape(columnName: String): String = s"`$columnName`" + // TODO use header.entityVars ? private def find[T <: ReturnItem](lookup: Var): T = relationalTable.header.idExpressions(lookup) .collectFirst { case expr if expr.withoutType == lookup.withoutType => expr } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index d510dc5bda383..c50fbca2ccf9a 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -35,8 +35,8 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe relationships: Seq[RelationshipDataFrame] ): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") - val nodeTables = nodes.map { nodeDataFrame => SparkNodeTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } - val relTables = relationships.map { relDataFrame => SparkRelationshipTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } + val nodeTables = nodes.map { nodeDataFrame => SparkNodeTable.create(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } + val relTables = relationships.map { relDataFrame => SparkRelationshipTable.create(relDataFrame.toRelationshipMapping, relDataFrame.df) } RelationalGraphAdapter(this, graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*)) } @@ -53,5 +53,3 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe SparkCypherResult(relationalGraph.cypher(query).records, relationalGraph.schema) } } - - diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala index 9b9eaa6aba14a..45a2278cdcfb4 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala @@ -4,7 +4,7 @@ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.opencypher.okapi.api.io.conversion.NodeMapping import org.opencypher.okapi.relational.api.io.NodeTable -case class SparkNodeTable(override val mapping: NodeMapping, override val table: DataFrameTable) +case class SparkNodeTable(mapping: NodeMapping, table: DataFrameTable) extends NodeTable(mapping, table) with RecordBehaviour { override type Records = SparkNodeTable @@ -13,4 +13,26 @@ case class SparkNodeTable(override val mapping: NodeMapping, override val table: table.cache() this } + +} + + +object SparkNodeTable { + + def create(mapping: NodeMapping, table: DataFrameTable): SparkNodeTable = { + val columnNames = table.df.columns + val columnRenames = columnNames.zip(columnNames.map(escape)).toMap + val escapedTable = table.df.withColumnsRenamed(columnRenames) + val escapedMapping = mapping match { + case NodeMapping(id, labels, optionalLabels, properties) => + NodeMapping(escape(id), labels, optionalLabels.mapValues(escape), properties.mapValues(escape)) + } + SparkNodeTable(escapedMapping, escapedTable) + } + + // TODO: Ensure that there are no conflicts with existing column names + private def escape(columnName: String): String = { + columnName.replaceAll("\\.", "_DOT_") + } + } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala index 2d2f603e5c23d..35018b255a531 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala @@ -2,9 +2,10 @@ package org.apache.spark.graph.cypher import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.opencypher.okapi.api.io.conversion.RelationshipMapping +import org.opencypher.okapi.impl.exception.UnsupportedOperationException import org.opencypher.okapi.relational.api.io.RelationshipTable -case class SparkRelationshipTable(override val mapping: RelationshipMapping, override val table: DataFrameTable) +case class SparkRelationshipTable(mapping: RelationshipMapping, table: DataFrameTable) extends RelationshipTable(mapping, table) with RecordBehaviour { override type Records = SparkRelationshipTable @@ -14,3 +15,25 @@ case class SparkRelationshipTable(override val mapping: RelationshipMapping, ove this } } + +object SparkRelationshipTable { + + def create(mapping: RelationshipMapping, table: DataFrameTable): SparkRelationshipTable = { + val columnNames = table.df.columns + val columnRenames = columnNames.zip(columnNames.map(escape)).toMap + val escapedTable = table.df.withColumnsRenamed(columnRenames) + val escapedMapping = mapping match { + case RelationshipMapping(id, start, end, relType@Left(_), properties) => + RelationshipMapping(escape(id), escape(start), escape(end), relType, properties.mapValues(escape)) + case _ => + throw UnsupportedOperationException("Relationship types in a string column") + } + SparkRelationshipTable(escapedMapping, escapedTable) + } + + // TODO: Ensure that there are no conflicts with existing column names + private def escape(columnName: String): String = { + columnName.replaceAll("\\.", "_DOT_") + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index b557d7eaf1e11..e66015b9f3a2b 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -286,15 +286,16 @@ object SparkTable { df.schema.fields(df.schema.fieldIndex(columnName)) } - def safeRenameColumns(renamings: Map[String, String]): DataFrame = { - if (renamings.isEmpty || renamings.forall { case (oldColumn, newColumn) => oldColumn == newColumn }) { + def safeRenameColumns(renames: Map[String, String]): DataFrame = { + val actualRenames = renames.filter { case (oldCol, newCol) => oldCol != newCol } + if (actualRenames.isEmpty) { df } else { - renamings.foreach { case (oldName, newName) => require(!df.columns.contains(newName), + actualRenames.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 renamings.contains(col) => renamings(col) + case col if actualRenames.contains(col) => actualRenames(col) case col => col } df.toDF(newColumns: _*) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index e9d55c9ba7214..8fda4e3f999eb 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -26,8 +26,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } - // TODO: requires fixing escaping in CAPS - ignore("create property graph from query results") { + test("create property graph from query results") { val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), "Alice", 42), Tuple3(id(1), "Bob", 23), Tuple3(id(2), "Eve", 19))).toDF("id", "name", "age") val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") @@ -46,9 +45,9 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { |RETURN p, o, k |""".stripMargin) - val berkeleyStudents: Seq[NodeDataFrame] = result.nodeDataFrame("p") - val berkeleyStudentFriends: Seq[NodeDataFrame] = result.nodeDataFrame("o") - val knows: Seq[RelationshipDataFrame] = result.relationshipDataFrame("k") + val berkeleyStudents: Seq[NodeDataFrame] = result.nodeDataFrames("p") + val berkeleyStudentFriends: Seq[NodeDataFrame] = result.nodeDataFrames("o") + val knows: Seq[RelationshipDataFrame] = result.relationshipDataFrames("k") val berkeleyGraph: PropertyGraph = cypherEngine.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala index 28e862a149735..1a32b7fe9b90c 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala @@ -85,7 +85,7 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { val encodedRecords = encodeIdColumns(records, nodeMapping) - SparkNodeTable(nodeMapping, encodedRecords) + SparkNodeTable.create(nodeMapping, encodedRecords) } val relScans = schema.relationshipTypes.map { relType => @@ -116,7 +116,7 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { val encodedRecords = encodeIdColumns(records, relationshipMapping) - SparkRelationshipTable(relationshipMapping, encodedRecords) + SparkRelationshipTable.create(relationshipMapping, encodedRecords) } new ScanGraph(nodeScans.toSeq ++ relScans, schema) From 7bed1f9f9a7c77093cbc0bc25d057fd492de7e69 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 25 Feb 2019 15:15:21 +0000 Subject: [PATCH 014/123] Adapt to changes in okapi-relational Co-authored-by: Martin Junghanns --- .../org/apache/spark/graph/cypher/SparkCypherResult.scala | 2 +- .../scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index 66c3adf89ce84..6ac7979764944 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -19,7 +19,7 @@ case class SparkCypherResult( private val header: RecordHeader = relationalTable.header - private lazy val hasMultipleEntities = header.entityVars.size > 1 + private lazy val hasMultipleEntities: Boolean = header.entityVars.size > 1 // TODO: Error handling override def nodeDataFrames(varName: String): Seq[NodeDataFrame] = { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 8fda4e3f999eb..fc9c74a2d3cce 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -22,6 +22,8 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val relationshipFrame: RelationshipDataFrame = RelationshipDataFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) + graph.nodes.show() + graph.relationships.show() graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } @@ -37,6 +39,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val studyAtDataFrame: RelationshipDataFrame = RelationshipDataFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") val graph: PropertyGraph = cypherEngine.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) + val result: CypherResult = graph.cypher( """ |MATCH (p:Student)-[:STUDY_AT]->(u:University), From 3a66462e8b4bc767310eb5dbb0bea9df5c4558a3 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 25 Feb 2019 16:06:55 +0000 Subject: [PATCH 015/123] Clearer semantics for column access via header It is only legal to convert an expression when the header contains it. If the header contains an expression, but the physical table does not contain the corresponding column, then the expression is converted to a null literal. Co-authored-by: Martin Junghanns --- .../cypher/conversions/ExprConversions.scala | 90 +++++++------------ .../cypher/tck/SparkCypherTckSuite.scala | 3 +- 2 files changed, 33 insertions(+), 60 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala index 7cc9db47d1f91..cc6ff701b75be 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -31,10 +31,6 @@ object ExprConversions { implicit class RichExpression(expr: Expr) { - def verify(implicit header: RecordHeader): Unit = { - if (header.expressionsFor(expr).isEmpty) throw IllegalStateException(s"Expression $expr not in header:\n${header.pretty}") - } - /** * This is possible without violating Cypher semantics because * - Spark SQL returns null when comparing across types (from initial investigation) @@ -45,13 +41,13 @@ object ExprConversions { comparator(lhs.asSparkSQLExpr)(rhs.asSparkSQLExpr) } - def lt(c: Column): Column => Column = c < _ + def lt(column: Column): Column => Column = column < _ - def lteq(c: Column): Column => Column = c <= _ + def lteq(column: Column): Column => Column = column <= _ - def gt(c: Column): Column => Column = c > _ + def gt(column: Column): Column => Column = column > _ - def gteq(c: Column): Column => Column = c >= _ + def gteq(column: Column): Column => Column = column >= _ /** * Attempts to create a Spark SQL expression from the CAPS expression. @@ -65,7 +61,7 @@ object ExprConversions { expr match { - // context based lookups + // Context based lookups case p@Param(name) if p.cypherType.isInstanceOf[CTList] => parameters(name) match { case CypherList(l) => functions.array(l.unwrap.map(functions.lit): _*) @@ -93,21 +89,17 @@ object ExprConversions { NULL_LIT case _ => - verify - - df.col(header.column(expr)) + columnFor(expr) } - // direct column lookup - case _: Var | _: Param | _: HasLabel | _: HasType | _: StartNode | _: EndNode => - verify + case IsNull(e) => + e.asSparkSQLExpr.isNull - val colName = header.column(expr) - if (df.columns.contains(colName)) { - df.col(colName) - } else { - NULL_LIT - } + case IsNotNull(e) => + e.asSparkSQLExpr.isNotNull + + case _: Var | _: Param | _: HasLabel | _: HasType | _: StartNode | _: EndNode => + columnFor(expr) case AliasExpr(innerExpr, _) => innerExpr.asSparkSQLExpr @@ -159,8 +151,6 @@ object ExprConversions { // predicates case Equals(e1, e2) => e1.asSparkSQLExpr === e2.asSparkSQLExpr case Not(e) => !e.asSparkSQLExpr - case IsNull(e) => e.asSparkSQLExpr.isNull - case IsNotNull(e) => e.asSparkSQLExpr.isNotNull case Size(e) => val col = e.asSparkSQLExpr e.cypherType match { @@ -174,11 +164,9 @@ object ExprConversions { case other => throw NotImplementedException(s"size() on values of type $other") } - case Ands(exprs) => - exprs.map(_.asSparkSQLExpr).foldLeft(TRUE_LIT)(_ && _) + case Ands(exprs) => exprs.map(_.asSparkSQLExpr).foldLeft(TRUE_LIT)(_ && _) - case Ors(exprs) => - exprs.map(_.asSparkSQLExpr).foldLeft(FALSE_LIT)(_ || _) + case Ors(exprs) => exprs.map(_.asSparkSQLExpr).foldLeft(FALSE_LIT)(_ || _) case In(lhs, rhs) => if (rhs.cypherType == CTNull || lhs.cypherType == CTNull) { @@ -194,12 +182,9 @@ object ExprConversions { case GreaterThanOrEqual(lhs, rhs) => compare(gteq, lhs, rhs) case GreaterThan(lhs, rhs) => compare(gt, lhs, rhs) - case StartsWith(lhs, rhs) => - lhs.asSparkSQLExpr.startsWith(rhs.asSparkSQLExpr) - case EndsWith(lhs, rhs) => - lhs.asSparkSQLExpr.endsWith(rhs.asSparkSQLExpr) - case Contains(lhs, rhs) => - lhs.asSparkSQLExpr.contains(rhs.asSparkSQLExpr) + case StartsWith(lhs, rhs) => lhs.asSparkSQLExpr.startsWith(rhs.asSparkSQLExpr) + case EndsWith(lhs, rhs) => lhs.asSparkSQLExpr.endsWith(rhs.asSparkSQLExpr) + case Contains(lhs, rhs) => lhs.asSparkSQLExpr.contains(rhs.asSparkSQLExpr) case RegexMatch(prop, Param(name)) => val regex: String = parameters(name).unwrap.toString @@ -235,36 +220,13 @@ object ExprConversions { case Subtract(lhs, rhs) if lhs.cypherType.material.subTypeOf(CTDate).isTrue && rhs.cypherType.material.subTypeOf(CTDuration).isTrue => TemporalUdfs.dateSubtract(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) - case Subtract(lhs, rhs) => - lhs.asSparkSQLExpr - rhs.asSparkSQLExpr + case Subtract(lhs, rhs) => lhs.asSparkSQLExpr - rhs.asSparkSQLExpr case Multiply(lhs, rhs) => lhs.asSparkSQLExpr * rhs.asSparkSQLExpr case div@Divide(lhs, rhs) => (lhs.asSparkSQLExpr / rhs.asSparkSQLExpr).cast(div.cypherType.getSparkType) - // Id functions - - case Id(e) => e.asSparkSQLExpr - - case PrefixId(idExpr, prefix) => -// idExpr.asSparkSQLExpr.addPrefix(functions.lit(prefix)) - ??? - - case ToId(e) => - e.cypherType.material match { - // TODO: Remove this call; we shouldn't have nodes or rels as concrete types here - case _: CTNode | _: CTRelationship => - e.asSparkSQLExpr - case CTInteger => -// e.asSparkSQLExpr.encodeLongAsCAPSId - ??? - case CTIdentity => - e.asSparkSQLExpr - case other => - throw IllegalArgumentException("a type that may be converted to an ID", other) - } - // Functions - case _: MonotonicallyIncreasingId => functions.monotonically_increasing_id() + case Id(e) => e.asSparkSQLExpr case Exists(e) => e.asSparkSQLExpr.isNotNull case Labels(e) => e.cypherType match { @@ -464,6 +426,18 @@ object ExprConversions { } } + private def columnFor(expr: Expr)(implicit header: RecordHeader, df: DataFrame): Column = { + val columnName = header.getColumn(expr).getOrElse(throw IllegalArgumentException( + expected = s"Expression in ${header.expressions.mkString("[", ", ", "]")}", + actual = expr) + ) + if (df.columns.contains(columnName)) { + df.col(columnName) + } else { + NULL_LIT + } + } + private def toSparkLiteral(value: Any): Column = value match { case map: Map[_, _] => val columns = map.map { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala index 3a981a06cd5d3..c849189c25f2b 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -28,7 +28,7 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { private val scenarios = ScenariosFor(failingBlacklist, temporalBlacklist, wontFixBlacklistFile, failureReportingBlacklistFile) forAll(scenarios.whiteList) { scenario => - test(s"[${graphFactory.name}, ${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { + test(s"[${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute() } } @@ -105,4 +105,3 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { .foreach(scenario => scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute()) } } - From f347238f6160865979bb053a7fb2effa0d039708 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 26 Feb 2019 16:43:35 +0100 Subject: [PATCH 016/123] Adapt to entity mapping changes in okapi Co-authored-by: Philip Stutz --- .../graph/cypher/SparkCypherSession.scala | 6 +-- .../spark/graph/cypher/SparkEntityTable.scala | 20 ++++++++++ .../spark/graph/cypher/SparkNodeTable.scala | 38 ------------------ .../graph/cypher/SparkRelationshipTable.scala | 39 ------------------- .../cypher/adapters/MappingAdapter.scala | 19 ++++++--- .../spark/graph/cypher/BasicMatchSuite.scala | 2 +- .../construction/ScanGraphFactory.scala | 23 ++++++----- 7 files changed, 50 insertions(+), 97 deletions(-) create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala delete mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala delete mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index c50fbca2ccf9a..366622ec75a08 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -4,7 +4,7 @@ import org.apache.spark.graph.api._ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.SparkSession import org.opencypher.okapi.impl.exception.IllegalArgumentException import org.opencypher.okapi.relational.api.graph.{RelationalCypherGraph, RelationalCypherGraphFactory, RelationalCypherSession} import org.opencypher.okapi.relational.api.planning.RelationalCypherResult @@ -35,8 +35,8 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe relationships: Seq[RelationshipDataFrame] ): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") - val nodeTables = nodes.map { nodeDataFrame => SparkNodeTable.create(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } - val relTables = relationships.map { relDataFrame => SparkRelationshipTable.create(relDataFrame.toRelationshipMapping, relDataFrame.df) } + val nodeTables = nodes.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } + val relTables = relationships.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } RelationalGraphAdapter(this, graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*)) } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala new file mode 100644 index 0000000000000..4dcb1f3956853 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala @@ -0,0 +1,20 @@ +package org.apache.spark.graph.cypher + +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.opencypher.okapi.api.io.conversion.EntityMapping +import org.opencypher.okapi.relational.api.io.EntityTable + +case class SparkEntityTable( + override val mapping: EntityMapping, + override val table: DataFrameTable +) extends EntityTable[DataFrameTable] with RecordBehaviour { + + override type Records = SparkEntityTable + + private[spark] def records(implicit cypherEngine: SparkCypherSession): SparkCypherRecords = cypherEngine.records.fromEntityTable(entityTable = this) + + override def cache(): SparkEntityTable = { + table.cache() + this + } +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala deleted file mode 100644 index 45a2278cdcfb4..0000000000000 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkNodeTable.scala +++ /dev/null @@ -1,38 +0,0 @@ -package org.apache.spark.graph.cypher - -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.opencypher.okapi.api.io.conversion.NodeMapping -import org.opencypher.okapi.relational.api.io.NodeTable - -case class SparkNodeTable(mapping: NodeMapping, table: DataFrameTable) - extends NodeTable(mapping, table) with RecordBehaviour { - - override type Records = SparkNodeTable - - override def cache(): SparkNodeTable = { - table.cache() - this - } - -} - - -object SparkNodeTable { - - def create(mapping: NodeMapping, table: DataFrameTable): SparkNodeTable = { - val columnNames = table.df.columns - val columnRenames = columnNames.zip(columnNames.map(escape)).toMap - val escapedTable = table.df.withColumnsRenamed(columnRenames) - val escapedMapping = mapping match { - case NodeMapping(id, labels, optionalLabels, properties) => - NodeMapping(escape(id), labels, optionalLabels.mapValues(escape), properties.mapValues(escape)) - } - SparkNodeTable(escapedMapping, escapedTable) - } - - // TODO: Ensure that there are no conflicts with existing column names - private def escape(columnName: String): String = { - columnName.replaceAll("\\.", "_DOT_") - } - -} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala deleted file mode 100644 index 35018b255a531..0000000000000 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkRelationshipTable.scala +++ /dev/null @@ -1,39 +0,0 @@ -package org.apache.spark.graph.cypher - -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.opencypher.okapi.api.io.conversion.RelationshipMapping -import org.opencypher.okapi.impl.exception.UnsupportedOperationException -import org.opencypher.okapi.relational.api.io.RelationshipTable - -case class SparkRelationshipTable(mapping: RelationshipMapping, table: DataFrameTable) - extends RelationshipTable(mapping, table) with RecordBehaviour { - - override type Records = SparkRelationshipTable - - override def cache(): SparkRelationshipTable = { - table.cache() - this - } -} - -object SparkRelationshipTable { - - def create(mapping: RelationshipMapping, table: DataFrameTable): SparkRelationshipTable = { - val columnNames = table.df.columns - val columnRenames = columnNames.zip(columnNames.map(escape)).toMap - val escapedTable = table.df.withColumnsRenamed(columnRenames) - val escapedMapping = mapping match { - case RelationshipMapping(id, start, end, relType@Left(_), properties) => - RelationshipMapping(escape(id), escape(start), escape(end), relType, properties.mapValues(escape)) - case _ => - throw UnsupportedOperationException("Relationship types in a string column") - } - SparkRelationshipTable(escapedMapping, escapedTable) - } - - // TODO: Ensure that there are no conflicts with existing column names - private def escape(columnName: String): String = { - columnName.replaceAll("\\.", "_DOT_") - } - -} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala index 83c19057d38b8..132821641fd39 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala @@ -1,18 +1,25 @@ package org.apache.spark.graph.cypher.adapters import org.apache.spark.graph.api.{NodeDataFrame, RelationshipDataFrame} -import org.opencypher.okapi.api.io.conversion.{NodeMapping, RelationshipMapping} +import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMappingBuilder, RelationshipMappingBuilder} object MappingAdapter { implicit class RichNodeDataFrame(val nodeDf: NodeDataFrame) extends AnyVal { - def toNodeMapping: NodeMapping = - NodeMapping(nodeDf.idColumn, nodeDf.labels, propertyMapping = nodeDf.properties) - + def toNodeMapping: EntityMapping = NodeMappingBuilder + .on(nodeDf.idColumn) + .withImpliedLabels(nodeDf.labels.toSeq: _*) + .withPropertyKeyMappings(nodeDf.properties.toSeq:_*) + .build } implicit class RichRelationshipDataFrame(val relDf: RelationshipDataFrame) extends AnyVal { - def toRelationshipMapping: RelationshipMapping = - RelationshipMapping(relDf.idColumn, relDf.sourceIdColumn, relDf.targetIdColumn, Left(relDf.relationshipType), relDf.properties) + def toRelationshipMapping: EntityMapping = RelationshipMappingBuilder + .on(relDf.idColumn) + .withSourceStartNodeKey(relDf.sourceIdColumn) + .withSourceEndNodeKey(relDf.targetIdColumn) + .withRelType(relDf.relationshipType) + .withPropertyKeyMappings(relDf.properties.toSeq: _*) + .build } } diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index fc9c74a2d3cce..bc859ac02f887 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -28,7 +28,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } - test("create property graph from query results") { + ignore("create property graph from query results") { val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), "Alice", 42), Tuple3(id(1), "Bob", 23), Tuple3(id(2), "Eve", 19))).toDF("id", "name", "age") val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala index 1a32b7fe9b90c..693b83f9b2e0b 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala @@ -5,10 +5,11 @@ import java.time.{LocalDate, LocalDateTime} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.conversions.TemporalConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ -import org.apache.spark.graph.cypher.{SparkCypherSession, SparkNodeTable, SparkRelationshipTable} +import org.apache.spark.graph.cypher.{SparkCypherSession, SparkEntityTable} import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row} -import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMapping, RelationshipMapping} +import org.opencypher.okapi.api.graph.Pattern +import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMappingBuilder, RelationshipMappingBuilder} import org.opencypher.okapi.api.schema.PropertyKeys.PropertyKeys import org.opencypher.okapi.impl.exception.IllegalArgumentException import org.opencypher.okapi.impl.temporal.Duration @@ -21,8 +22,7 @@ import scala.collection.JavaConverters._ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { def encodeIdColumns(df: DataFrame, mapping: EntityMapping): DataFrame = { - - val idCols = mapping.idKeys.map { columnName => + 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) @@ -35,7 +35,7 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { ) } } - val remainingCols = mapping.allSourceKeys.filterNot(mapping.idKeys.contains).map(df.col) + val remainingCols = mapping.allSourceKeys.filterNot(mapping.allSourceIdKeys.contains).map(df.col) val colsToSelect = idCols ++ remainingCols df.select(colsToSelect: _*) } @@ -50,8 +50,9 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { val tableEntityStartNodeKey = "___source" val tableEntityEndNodeKey = "___target" - override def apply(propertyGraph: InMemoryTestGraph) + 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 => @@ -78,14 +79,15 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { val records = sparkCypher.sparkSession.createDataFrame(rows.asJava, structType).toDF(header: _*) - val nodeMapping = NodeMapping + val nodeMapping = NodeMappingBuilder .on(tableEntityIdKey) .withImpliedLabels(labels.toSeq: _*) .withPropertyKeys(propKeys.keys.toSeq: _*) + .build val encodedRecords = encodeIdColumns(records, nodeMapping) - SparkNodeTable.create(nodeMapping, encodedRecords) + SparkEntityTable(nodeMapping, encodedRecords) } val relScans = schema.relationshipTypes.map { relType => @@ -107,16 +109,17 @@ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { val records = sparkCypher.sparkSession.createDataFrame(rows.asJava, structType).toDF(header: _*) - val relationshipMapping = RelationshipMapping + val relationshipMapping = RelationshipMappingBuilder .on(tableEntityIdKey) .from(tableEntityStartNodeKey) .to(tableEntityEndNodeKey) .relType(relType) .withPropertyKeys(propKeys.keys.toSeq: _*) + .build val encodedRecords = encodeIdColumns(records, relationshipMapping) - SparkRelationshipTable.create(relationshipMapping, encodedRecords) + SparkEntityTable(relationshipMapping, encodedRecords) } new ScanGraph(nodeScans.toSeq ++ relScans, schema) From 417dc8e9ac1269be139b12c14e0cc4c137fcd942 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 26 Feb 2019 17:32:23 +0100 Subject: [PATCH 017/123] Enable round trip example MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling Co-authored-by: Philip Stutz --- .../scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index bc859ac02f887..fc9c74a2d3cce 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -28,7 +28,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } - ignore("create property graph from query results") { + test("create property graph from query results") { val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), "Alice", 42), Tuple3(id(1), "Bob", 23), Tuple3(id(2), "Eve", 19))).toDF("id", "name", "age") val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") From ee968efc611e347dd25f54bc1494cc64db59c371 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 26 Feb 2019 17:55:22 +0100 Subject: [PATCH 018/123] Rename Node/RelationshipDataFrame to Node/RelationshipFrame MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling Co-authored-by: Philip Stutz --- .../apache/spark/graph/api/CypherEngine.scala | 4 ++-- .../apache/spark/graph/api/CypherResult.scala | 4 ++-- ...ataFrame.scala => GraphElementFrame.scala} | 10 ++++----- .../graph/cypher/SparkCypherResult.scala | 10 ++++----- .../graph/cypher/SparkCypherSession.scala | 4 ++-- .../cypher/adapters/MappingAdapter.scala | 6 ++--- .../spark/graph/cypher/BasicMatchSuite.scala | 22 +++++++++---------- 7 files changed, 30 insertions(+), 30 deletions(-) rename graph/api/src/main/scala/org/apache/spark/graph/api/{GraphElementDataFrame.scala => GraphElementFrame.scala} (74%) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala index 7e9af74f5e111..443026274597a 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala @@ -4,7 +4,7 @@ trait CypherEngine { def cypher(graph: PropertyGraph, query: String): CypherResult def createGraph( - nodes: Seq[NodeDataFrame], - relationships: Seq[RelationshipDataFrame] = Seq.empty + nodes: Seq[NodeFrame], + relationships: Seq[RelationshipFrame] = Seq.empty ): PropertyGraph } 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 index d690e499eca04..aef325efbf1f9 100644 --- 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 @@ -5,7 +5,7 @@ import org.apache.spark.sql.DataFrame trait CypherResult { def df: DataFrame - def nodeDataFrames(varName: String): Seq[NodeDataFrame] + def nodeDataFrames(varName: String): Seq[NodeFrame] - def relationshipDataFrames(varName: String): Seq[RelationshipDataFrame] + def relationshipDataFrames(varName: String): Seq[RelationshipFrame] } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala similarity index 74% rename from graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 510ad44e33293..2131f0501874d 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -2,7 +2,7 @@ package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame -trait GraphElementDataFrame { +trait GraphElementFrame { def df: DataFrame @@ -11,18 +11,18 @@ trait GraphElementDataFrame { def properties: Map[String, String] } -case class NodeDataFrame( +case class NodeFrame( df: DataFrame, idColumn: String, labels: Set[String] = Set.empty, properties: Map[String, String] = Map.empty -) extends GraphElementDataFrame +) extends GraphElementFrame -case class RelationshipDataFrame( +case class RelationshipFrame( df: DataFrame, idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String, properties: Map[String, String] = Map.empty -) extends GraphElementDataFrame +) extends GraphElementFrame diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index 6ac7979764944..bfacf6d4228dd 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -1,6 +1,6 @@ package org.apache.spark.graph.cypher -import org.apache.spark.graph.api.{CypherResult, NodeDataFrame, RelationshipDataFrame} +import org.apache.spark.graph.api.{CypherResult, NodeFrame, RelationshipFrame} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.sql.{DataFrame, functions} import org.opencypher.okapi.api.schema.Schema @@ -22,7 +22,7 @@ case class SparkCypherResult( private lazy val hasMultipleEntities: Boolean = header.entityVars.size > 1 // TODO: Error handling - override def nodeDataFrames(varName: String): Seq[NodeDataFrame] = { + override def nodeDataFrames(varName: String): Seq[NodeFrame] = { val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) val idColumn = header.column(nodeVar) @@ -50,12 +50,12 @@ case class SparkCypherResult( val distinctDf = if (hasMultipleEntities) labelCombinationDf.dropDuplicates(idColumn) else labelCombinationDf - NodeDataFrame(distinctDf, idColumn, labels, properties) + NodeFrame(distinctDf, idColumn, labels, properties) } } // TODO: Error handling - override def relationshipDataFrames(varName: String): Seq[RelationshipDataFrame] = { + override def relationshipDataFrames(varName: String): Seq[RelationshipFrame] = { val relVar: RelationshipVar = find(RelationshipVar(varName)(CTRelationship)) val idColumn = header.column(relVar) @@ -79,7 +79,7 @@ case class SparkCypherResult( val distinctDf = if (hasMultipleEntities) relTypeDf.dropDuplicates(idColumn) else relTypeDf - RelationshipDataFrame(distinctDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + RelationshipFrame(distinctDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 366622ec75a08..5e77812b37677 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -31,8 +31,8 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe override def entityTables: RelationalEntityTableFactory[DataFrameTable] = ??? override def createGraph( - nodes: Seq[NodeDataFrame], - relationships: Seq[RelationshipDataFrame] + nodes: Seq[NodeFrame], + relationships: Seq[RelationshipFrame] ): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") val nodeTables = nodes.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala index 132821641fd39..b9dfc60f8ccd5 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala @@ -1,11 +1,11 @@ package org.apache.spark.graph.cypher.adapters -import org.apache.spark.graph.api.{NodeDataFrame, RelationshipDataFrame} +import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMappingBuilder, RelationshipMappingBuilder} object MappingAdapter { - implicit class RichNodeDataFrame(val nodeDf: NodeDataFrame) extends AnyVal { + implicit class RichNodeDataFrame(val nodeDf: NodeFrame) extends AnyVal { def toNodeMapping: EntityMapping = NodeMappingBuilder .on(nodeDf.idColumn) .withImpliedLabels(nodeDf.labels.toSeq: _*) @@ -13,7 +13,7 @@ object MappingAdapter { .build } - implicit class RichRelationshipDataFrame(val relDf: RelationshipDataFrame) extends AnyVal { + implicit class RichRelationshipDataFrame(val relDf: RelationshipFrame) extends AnyVal { def toRelationshipMapping: EntityMapping = RelationshipMappingBuilder .on(relDf.idColumn) .withSourceStartNodeKey(relDf.sourceIdColumn) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index fc9c74a2d3cce..160a84721c5bf 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -1,14 +1,14 @@ package org.apache.spark.graph.cypher import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.api.{CypherResult, NodeDataFrame, PropertyGraph, RelationshipDataFrame} +import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} import org.apache.spark.sql.DataFrame class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("match single node pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") - val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) + val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) @@ -18,8 +18,8 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("match simple pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") - val nodeDataFrame: NodeDataFrame = NodeDataFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) - val relationshipFrame: RelationshipDataFrame = RelationshipDataFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) + val relationshipFrame: RelationshipFrame = RelationshipFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) graph.nodes.show() @@ -33,10 +33,10 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(2), id(0), id(2)), Tuple3(id(3), id(1), id(3)), Tuple3(id(4), id(2), id(2)))).toDF("id", "source", "target") - val personDataFrame: NodeDataFrame = NodeDataFrame(df = personData, idColumn = "id", labels = Set("Student"), properties = Map("name" -> "name", "age" -> "age")) - val universityDataFrame: NodeDataFrame = NodeDataFrame(df = universityData, idColumn = "id", labels = Set("University"), properties = Map("title" -> "title")) - val knowsDataFrame: RelationshipDataFrame = RelationshipDataFrame(df = knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val studyAtDataFrame: RelationshipDataFrame = RelationshipDataFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") + val personDataFrame: NodeFrame = NodeFrame(df = personData, idColumn = "id", labels = Set("Student"), properties = Map("name" -> "name", "age" -> "age")) + val universityDataFrame: NodeFrame = NodeFrame(df = universityData, idColumn = "id", labels = Set("University"), properties = Map("title" -> "title")) + val knowsDataFrame: RelationshipFrame = RelationshipFrame(df = knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val studyAtDataFrame: RelationshipFrame = RelationshipFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") val graph: PropertyGraph = cypherEngine.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) @@ -48,9 +48,9 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { |RETURN p, o, k |""".stripMargin) - val berkeleyStudents: Seq[NodeDataFrame] = result.nodeDataFrames("p") - val berkeleyStudentFriends: Seq[NodeDataFrame] = result.nodeDataFrames("o") - val knows: Seq[RelationshipDataFrame] = result.relationshipDataFrames("k") + val berkeleyStudents: Seq[NodeFrame] = result.nodeDataFrames("p") + val berkeleyStudentFriends: Seq[NodeFrame] = result.nodeDataFrames("o") + val knows: Seq[RelationshipFrame] = result.relationshipDataFrames("k") val berkeleyGraph: PropertyGraph = cypherEngine.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() From fd8d968562de39ba6326589c9f933cd4727e9404 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 26 Feb 2019 18:09:14 +0100 Subject: [PATCH 019/123] Adapt CypherResult to GraphElementFrame name changes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling Co-authored-by: Philip Stutz --- .../scala/org/apache/spark/graph/api/CypherResult.scala | 4 ++-- .../org/apache/spark/graph/cypher/SparkCypherResult.scala | 4 ++-- .../org/apache/spark/graph/cypher/BasicMatchSuite.scala | 6 +++--- 3 files changed, 7 insertions(+), 7 deletions(-) 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 index aef325efbf1f9..ee38a7302d0d2 100644 --- 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 @@ -5,7 +5,7 @@ import org.apache.spark.sql.DataFrame trait CypherResult { def df: DataFrame - def nodeDataFrames(varName: String): Seq[NodeFrame] + def nodeFrames(varName: String): Seq[NodeFrame] - def relationshipDataFrames(varName: String): Seq[RelationshipFrame] + def relationshipFrames(varName: String): Seq[RelationshipFrame] } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index bfacf6d4228dd..ebeac1c20ae4c 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -22,7 +22,7 @@ case class SparkCypherResult( private lazy val hasMultipleEntities: Boolean = header.entityVars.size > 1 // TODO: Error handling - override def nodeDataFrames(varName: String): Seq[NodeFrame] = { + override def nodeFrames(varName: String): Seq[NodeFrame] = { val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) val idColumn = header.column(nodeVar) @@ -55,7 +55,7 @@ case class SparkCypherResult( } // TODO: Error handling - override def relationshipDataFrames(varName: String): Seq[RelationshipFrame] = { + override def relationshipFrames(varName: String): Seq[RelationshipFrame] = { val relVar: RelationshipVar = find(RelationshipVar(varName)(CTRelationship)) val idColumn = header.column(relVar) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 160a84721c5bf..48db7bfdd3a87 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -48,9 +48,9 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { |RETURN p, o, k |""".stripMargin) - val berkeleyStudents: Seq[NodeFrame] = result.nodeDataFrames("p") - val berkeleyStudentFriends: Seq[NodeFrame] = result.nodeDataFrames("o") - val knows: Seq[RelationshipFrame] = result.relationshipDataFrames("k") + val berkeleyStudents: Seq[NodeFrame] = result.nodeFrames("p") + val berkeleyStudentFriends: Seq[NodeFrame] = result.nodeFrames("o") + val knows: Seq[RelationshipFrame] = result.relationshipFrames("k") val berkeleyGraph: PropertyGraph = cypherEngine.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() From 9c4786486ed8b1b45ee4bf7289fe73ba44e68f30 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Tue, 26 Feb 2019 17:27:51 +0000 Subject: [PATCH 020/123] Add constructor for GraphElementFrames that infers property mappings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../spark/graph/api/GraphElementFrame.scala | 36 +++++++++++++++++-- .../spark/graph/cypher/BasicMatchSuite.scala | 8 ++--- 2 files changed, 37 insertions(+), 7 deletions(-) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 2131f0501874d..2619c5c80afa8 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -11,18 +11,48 @@ trait GraphElementFrame { def properties: Map[String, String] } +object NodeFrame { + + def apply( + df: DataFrame, + idColumn: String, + labels: Set[String] = Set.empty + ): NodeFrame = { + val properties = (df.columns.toSet - idColumn) + .map(columnName => columnName -> columnName).toMap + NodeFrame(df, idColumn, labels, properties) + } + +} + case class NodeFrame( df: DataFrame, idColumn: String, - labels: Set[String] = Set.empty, - properties: Map[String, String] = Map.empty + labels: Set[String], + properties: Map[String, String] ) extends GraphElementFrame +object RelationshipFrame { + + def apply( + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String + ): RelationshipFrame = { + val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) + .map(columnName => columnName -> columnName).toMap + RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + } + +} + case class RelationshipFrame( df: DataFrame, idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String, - properties: Map[String, String] = Map.empty + properties: Map[String, String] ) extends GraphElementFrame diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 48db7bfdd3a87..6d951ae8d02fe 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -8,7 +8,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("match single node pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") - val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) + val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) @@ -18,7 +18,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("match simple pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") - val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person"), properties = Map("name" -> "name")) + val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) val relationshipFrame: RelationshipFrame = RelationshipFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) @@ -33,8 +33,8 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(2), id(0), id(2)), Tuple3(id(3), id(1), id(3)), Tuple3(id(4), id(2), id(2)))).toDF("id", "source", "target") - val personDataFrame: NodeFrame = NodeFrame(df = personData, idColumn = "id", labels = Set("Student"), properties = Map("name" -> "name", "age" -> "age")) - val universityDataFrame: NodeFrame = NodeFrame(df = universityData, idColumn = "id", labels = Set("University"), properties = Map("title" -> "title")) + val personDataFrame: NodeFrame = NodeFrame(df = personData, idColumn = "id", labels = Set("Student")) + val universityDataFrame: NodeFrame = NodeFrame(df = universityData, idColumn = "id", labels = Set("University")) val knowsDataFrame: RelationshipFrame = RelationshipFrame(df = knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") val studyAtDataFrame: RelationshipFrame = RelationshipFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") From 7d7be323224cc9de343f06af490a60caee3858b1 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Tue, 26 Feb 2019 18:56:12 +0000 Subject: [PATCH 021/123] Adapt branch-2.2 graph work to changes in branch-2.4 --- graph/cypher/pom.xml | 9 +++++++++ .../apache/spark/graph/cypher/SharedCypherContext.scala | 5 +++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index 4cb700c21af2a..c35afb71f3909 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -73,6 +73,7 @@ test-jar test + org.apache.spark spark-sql_${scala.binary.version} @@ -88,6 +89,14 @@ + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + + org.opencypher okapi-tck diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala index 22f2e9abfe709..c1c64b8026930 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala @@ -1,8 +1,9 @@ package org.apache.spark.graph.cypher -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.test.SharedSparkSession +import org.scalatest.Suite -trait SharedCypherContext extends SharedSQLContext { +trait SharedCypherContext extends SharedSparkSession { self: Suite => private var _cypherEngine: SparkCypherSession = _ From a171e4dbfa352a98131256a34819de6b495f9e2b Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Wed, 27 Feb 2019 08:31:13 +0000 Subject: [PATCH 022/123] Adapt blacklist due to changes in Spark 2.4 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Same changes as in CAPS. Co-authored-by: Max Kießling --- graph/cypher/src/test/resources/tck/failing_blacklist | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/graph/cypher/src/test/resources/tck/failing_blacklist b/graph/cypher/src/test/resources/tck/failing_blacklist index 0c02e4c472843..53f3c7a52477e 100644 --- a/graph/cypher/src/test/resources/tck/failing_blacklist +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -225,10 +225,10 @@ Feature "Aggregation": Scenario "`max()` over strings" Feature "Aggregation": Scenario "`max()` over mixed numeric values" Feature "Aggregation": Scenario "`min()` over strings" Feature "Aggregation": Scenario "`max()` over list 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 null when LHS and RHS both ultimately contain null, even if LHS and RHS are of different types (nested list and flat list)" Feature "ListOperations": Scenario "IN should return false when matching a number with a string - list version" -Feature "ListOperations": Scenario "IN should work with an empty list" Feature "ListOperations": Scenario "IN should return false when types of LHS and RHS don't match - singleton list" 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" @@ -272,7 +272,6 @@ Feature "ListOperations": Scenario "Equality between almost equal lists with nul 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 "IN should return true with previous null match, list version" Feature "ListOperations": Scenario "Collect and filter using a list comprehension" Feature "ListOperations": Scenario "Concatenating and returning the size of literal lists" Feature "ListOperations": Scenario "Concatenating lists of same type #1" \ No newline at end of file From ed7d8cc3aa0d2955587ea269561e02b7a273b6b7 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 27 Feb 2019 11:36:52 +0100 Subject: [PATCH 023/123] Add documentation for spark-graph-api MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling Co-authored-by: Philip Stutz --- .../apache/spark/graph/api/CypherEngine.scala | 10 ---- .../apache/spark/graph/api/CypherResult.scala | 42 +++++++++++++++ .../spark/graph/api/CypherSession.scala | 37 +++++++++++++ .../spark/graph/api/GraphElementFrame.scala | 52 +++++++++++++++++++ .../spark/graph/api/PropertyGraph.scala | 31 +++++++++-- .../graph/cypher/SparkCypherSession.scala | 20 ++++--- .../adapters/RelationalGraphAdapter.scala | 4 +- 7 files changed, 172 insertions(+), 24 deletions(-) delete mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala deleted file mode 100644 index 443026274597a..0000000000000 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/CypherEngine.scala +++ /dev/null @@ -1,10 +0,0 @@ -package org.apache.spark.graph.api - -trait CypherEngine { - def cypher(graph: PropertyGraph, query: String): CypherResult - - def createGraph( - nodes: Seq[NodeFrame], - relationships: Seq[RelationshipFrame] = Seq.empty - ): PropertyGraph -} 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 index ee38a7302d0d2..4fb9a4544a3e3 100644 --- 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 @@ -2,10 +2,52 @@ package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame +/** + * Result of a Cypher query. + * + * Wraps a [[DataFrame]] that contains the result rows. + */ trait CypherResult { + /** + * 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 the underscore (i.e. `_`), + * (e.g. `MATCH (n:Person) RETURN n` results in the columns `n`, `n:Person` and `n_name`). + */ def df: DataFrame + /** + * Extracts nodes that are specified as a return item. + * + * @example + * {{{ + * val result = graph.cypher("MATCH (n:Person)-[r:LIVES_IN]->(c:City) RETURN n, r, c") + * val nodeFrames = result.nodeFrames("n") + * }}} + * + * Returns a sequence of [[NodeFrame]]s where each single [[NodeFrame]] contains all distinct node entities that have at least label `:Person`. + * For example, if the graph contains nodes with label `:Person` and `:Person:Fireman` the sequence contains two [[NodeFrame]]s, one + * for each label combination. + * + * @param varName return item name of the node to extract + */ def nodeFrames(varName: String): Seq[NodeFrame] + /** + * Extracts relationships that are specified as a return item. + * + * @example + * {{{ + * val result = graph.cypher("MATCH (n:Person)-[r:LIVES_IN|WORKS_IN]->(c:City) RETURN n, r, c") + * val nodeFrames = result.nodeFrames("r") + * }}} + * + * Returns a sequence of two [[RelationshipFrame]]s, one that contains all distinct relationships with type `:LIVES_IN` and another for type `:WORKS_IN`. + * + * @param varName return item name of the relationships to extract + */ def relationshipFrames(varName: String): Seq[RelationshipFrame] } 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..bc718ee7fa8e6 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/CypherSession.scala @@ -0,0 +1,37 @@ +package org.apache.spark.graph.api + +import org.apache.spark.sql.SparkSession + +/** + * Allows for creating [[PropertyGraph]] instances and running Cypher-queries on them. + */ +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 + */ + def cypher(graph: PropertyGraph, query: String): CypherResult + + /** + * Executes a Cypher query on the given input graph. + * + * @param graph [[PropertyGraph]] on which the query is executed + * @param query Cypher query to execute + * @param parameters parameters used by the Cypher query + */ + def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult + + /** + * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. + * At least one [[NodeFrame]] has to be provided. + * + * @param nodes [[NodeFrame]]s that define the nodes in the graph + * @param relationships [[RelationshipFrame]]s that define the relationships in the graph + */ + def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 2619c5c80afa8..2b28005c6ff47 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -2,17 +2,40 @@ package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame +/** + * Describes how to map an input [[DataFrame]] to graph elements (i.e. nodes or relationships). + */ trait GraphElementFrame { + /** + * [[DataFrame]] containing element data. Each row represents a graph element. + */ def df: DataFrame + /** + * Name of the column that contains the graph element identifier. + * + * @note Column values need to be of [[org.apache.spark.sql.types.BinaryType]]. + */ def idColumn: String + /** + * Mapping from graph element property keys to the columns that contain the corresponding property values. + */ def properties: Map[String, String] } object NodeFrame { + /** + * Describes how to map an input [[DataFrame]] to nodes. + * + * All columns apart from the given `idColumn` are mapped to node properties. + * + * @param df [[DataFrame]] containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labels labels that are assigned to all nodes + */ def apply( df: DataFrame, idColumn: String, @@ -25,6 +48,14 @@ object NodeFrame { } +/** + * Describes how to map an input [[DataFrame]] to nodes. + * + * @param df [[DataFrame]] containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labels labels that are assigned to all nodes + * @param properties mapping from property keys to corresponding columns + */ case class NodeFrame( df: DataFrame, idColumn: String, @@ -34,6 +65,17 @@ case class NodeFrame( object RelationshipFrame { + /** + * Describes how to map an input [[DataFrame]] to relationships. + * + * All columns apart from the given identifier columns are mapped to relationship properties. + * + * @param df [[DataFrame]] 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 + */ def apply( df: DataFrame, idColumn: String, @@ -48,6 +90,16 @@ object RelationshipFrame { } +/** + * Describes how to map an input [[DataFrame]] to relationships. + * + * @param df [[DataFrame]] 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 properties mapping from property keys to corresponding columns + */ case class RelationshipFrame( df: DataFrame, idColumn: String, 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 index 2b4f3b7a12278..3ae8056cf99c4 100644 --- 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 @@ -2,14 +2,35 @@ package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame +/** + * A Property Graph as defined by the openCypher Property Graph Model. + * + * A graph is always tied to and managed by a [[CypherSession]]. The lifetime of a graph is bounded by the session lifetime. + * + * @see [[https://github.com/opencypher/openCypher/blob/master/docs/property-graph-model.adoc openCypher Property Graph Model]] + */ trait PropertyGraph { - def cypherEngine: CypherEngine - def cypher(query: String): CypherResult = - cypherEngine.cypher(this, query) + /** + * The session in which this graph is managed. + */ + 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 + * @param parameters parameters used by the Cypher query + */ + def cypher(query: String, parameters: Map[String, Any] = Map.empty): CypherResult = cypherSession.cypher(this, query, parameters) + + /** + * Returns a [[DataFrame]] that contains a row for each node in this graph. + */ def nodes: DataFrame - def vertices: DataFrame = nodes + + /** + * Returns a [[DataFrame]] that contains a row for each relationship in this graph. + */ def relationships: DataFrame - def edges: DataFrame = relationships } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 5e77812b37677..96e370c57587f 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -5,6 +5,7 @@ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter import org.apache.spark.sql.SparkSession +import org.opencypher.okapi.api.value.CypherValue.CypherMap import org.opencypher.okapi.impl.exception.IllegalArgumentException import org.opencypher.okapi.relational.api.graph.{RelationalCypherGraph, RelationalCypherGraphFactory, RelationalCypherSession} import org.opencypher.okapi.relational.api.planning.RelationalCypherResult @@ -14,7 +15,13 @@ object SparkCypherSession { def create(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) } -class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherEngine { +/** + * Default [[CypherSession]] implementation. + * + * This class is the main entry point for working with the spark-graph-cypher module. + * It wraps a [[SparkSession]] and allows to run Cypher queries over graphs represented as [[org.apache.spark.sql.DataFrame]]s. + */ +class SparkCypherSession(override val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherSession { override type Result = RelationalCypherResult[DataFrameTable] override type Records = SparkCypherRecords @@ -30,10 +37,7 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe override def entityTables: RelationalEntityTableFactory[DataFrameTable] = ??? - override def createGraph( - nodes: Seq[NodeFrame], - relationships: Seq[RelationshipFrame] - ): PropertyGraph = { + override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") val nodeTables = nodes.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } val relTables = relationships.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } @@ -41,7 +45,9 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe RelationalGraphAdapter(this, graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*)) } - override def cypher(graph: PropertyGraph, query: String): CypherResult = { + def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) + + override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { val relationalGraph = graph match { case RelationalGraphAdapter(_, relGraph) => relGraph case other => throw IllegalArgumentException( @@ -50,6 +56,6 @@ class SparkCypherSession(val sparkSession: SparkSession) extends RelationalCyphe ) } - SparkCypherResult(relationalGraph.cypher(query).records, relationalGraph.schema) + SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema) } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala index 601858b112c1b..98100e27ded01 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala @@ -1,12 +1,12 @@ package org.apache.spark.graph.cypher.adapters -import org.apache.spark.graph.api.{CypherEngine, PropertyGraph} +import org.apache.spark.graph.api.{CypherSession, PropertyGraph} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.sql.DataFrame import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph case class RelationalGraphAdapter( - cypherEngine: CypherEngine, + cypherSession: CypherSession, graph: RelationalCypherGraph[DataFrameTable]) extends PropertyGraph { override def nodes: DataFrame = graph.nodes("n").table.df From c1d72f65e32fc9329cc4e31e4580c7ede495fe9e Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Wed, 27 Feb 2019 10:49:21 +0000 Subject: [PATCH 024/123] Add default parameter for relationships in `createGraph` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Martin Junghanns Co-authored-by: Max Kießling --- .../org/apache/spark/graph/cypher/SparkCypherSession.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 96e370c57587f..c6f92f022a895 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -37,7 +37,7 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati override def entityTables: RelationalEntityTableFactory[DataFrameTable] = ??? - override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph = { + override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") val nodeTables = nodes.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } val relTables = relationships.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } From 579740008c7f74eff00e14f6305fda88b35f7d76 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 27 Feb 2019 14:12:17 +0100 Subject: [PATCH 025/123] Use Spark 2.4 features where possible and remove/inline helpers MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling Co-authored-by: Philip Stutz --- .../spark/graph/cypher/SparkTable.scala | 12 +--- .../cypher/conversions/ExprConversions.scala | 11 ++-- .../spark/graph/cypher/udfs/LegacyUdfs.scala | 58 +------------------ 3 files changed, 10 insertions(+), 71 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index e66015b9f3a2b..32a5bd30e240a 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -193,16 +193,8 @@ object SparkTable { } joinType match { - case CrossJoin => - df.crossJoin(other.df) - - case _ => - - // TODO: the join produced corrupt data when the previous operator was a cross. We work around that by using a - // subsequent select. This can be removed, once https://issues.apache.org/jira/browse/SPARK-23855 is solved or we - // upgrade to Spark 2.3.0 - val potentiallyCorruptedResult = df.safeJoin(other.df, joinCols, joinTypeString) - potentiallyCorruptedResult.select("*") + case CrossJoin => df.crossJoin(other.df) + case _ => df.safeJoin(other.df, joinCols, joinTypeString) } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala index cc6ff701b75be..4d12e04ab709c 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -4,6 +4,8 @@ import org.apache.spark.graph.cypher.conversions.TemporalConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ import org.apache.spark.graph.cypher.udfs.LegacyUdfs._ import org.apache.spark.graph.cypher.udfs.TemporalUdfs +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.expressions.{ArrayContains, StringTranslate} import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, functions} import org.opencypher.okapi.api.types._ @@ -174,7 +176,7 @@ object ExprConversions { } else { val element = lhs.asSparkSQLExpr val array = rhs.asSparkSQLExpr - array_contains(array, element) + new Column(ArrayContains(element.expr, array.expr)) } case LessThan(lhs, rhs) => compare(lt, lhs, rhs) @@ -318,9 +320,10 @@ object ExprConversions { case Range(from, to, maybeStep) => val stepCol = maybeStep.map(_.asSparkSQLExpr).getOrElse(ONE_LIT) - rangeUdf(from.asSparkSQLExpr, to.asSparkSQLExpr, stepCol) + functions.sequence(from.asSparkSQLExpr, to.asSparkSQLExpr, stepCol) - case Replace(original, search, replacement) => translateColumn(original.asSparkSQLExpr, search.asSparkSQLExpr, replacement.asSparkSQLExpr) + case Replace(original, search, replacement) => + new Column(StringTranslate(original.asSparkSQLExpr.expr, search.asSparkSQLExpr.expr, replacement.asSparkSQLExpr.expr)) case Substring(original, start, maybeLength) => val origCol = original.asSparkSQLExpr @@ -404,7 +407,7 @@ object ExprConversions { val containerCol = container.asSparkSQLExpr container.cypherType.material match { - case _: CTList | _: CTMap => containerCol.get(indexCol) + case _: CTList | _: CTMap => new Column(UnresolvedExtractValue(containerCol.expr, indexCol.expr)) case other => throw NotImplementedException(s"Accessing $other by index is not supported") } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala index 0eef092b38dc4..d08d0f9a064a1 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala @@ -1,59 +1,11 @@ package org.apache.spark.graph.cypher.udfs -import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue -import org.apache.spark.sql.catalyst.expressions.{ArrayContains, StringTranslate, XxHash64} import org.apache.spark.sql.expressions.UserDefinedFunction -import org.apache.spark.sql.functions.{monotonically_increasing_id, udf} +import org.apache.spark.sql.functions import org.apache.spark.sql.types.{ArrayType, StringType} -import org.apache.spark.sql.{Column, functions} object LegacyUdfs { - 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)) - } - - val rangeUdf: UserDefinedFunction = - udf[Array[Int], Int, Int, Int]((from: Int, to: Int, step: Int) => from.to(to, step).toArray) - - private[spark] val rowIdSpaceBitsUsedByMonotonicallyIncreasingId = 33 - - /** - * Configurable wrapper around `monotonically_increasing_id` - * - * @param partitionStartDelta Conceptually this number is added to the `partitionIndex` from which the Spark function - * `monotonically_increasing_id` starts assigning IDs. - */ - // TODO: Document inherited limitations with regard to the maximum number of rows per data frame - // TODO: Document the maximum number of partitions (before entering tag space) - def partitioned_id_assignment(partitionStartDelta: Int): Column = - monotonically_increasing_id() + (partitionStartDelta.toLong << rowIdSpaceBitsUsedByMonotonicallyIncreasingId) - - /** - * 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 array_append_long(array: Column, value: Column): Column = - appendLongUDF(array, value) - - private val appendLongUDF = - functions.udf(appendLong _) - - private def appendLong(array: Seq[Long], element: Long): Seq[Long] = - array :+ element - def get_rel_type(relTypeNames: Seq[String]): UserDefinedFunction = { val extractRelTypes = (booleanMask: Seq[Boolean]) => filterWithMask(relTypeNames)(booleanMask) functions.udf(extractRelTypes.andThen(_.headOption.orNull), StringType) @@ -74,13 +26,5 @@ object LegacyUdfs { dataToFilter.zip(values).collect { case (key, value) if value != null => key } - - /** - * Alternative version of {{{org.apache.spark.sql.functions.translate}}} that takes {{{org.apache.spark.sql.Column}}}s for search and replace strings. - */ - def translateColumn(src: Column, matchingString: Column, replaceString: Column): Column = { - new Column(StringTranslate(src.expr, matchingString.expr, replaceString.expr)) - } - } From 52c3d5ca56d82c03663dea1dbf1d7a5fb9a0b0a7 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 27 Feb 2019 15:49:38 +0100 Subject: [PATCH 026/123] Use CypherResult to create new Property Graph MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../spark/graph/api/CypherSession.scala | 11 ++++++++-- .../graph/cypher/SparkCypherResult.scala | 17 +++++----------- .../graph/cypher/SparkCypherSession.scala | 20 +++++++++++++++++++ .../spark/graph/cypher/BasicMatchSuite.scala | 5 ++++- 4 files changed, 38 insertions(+), 15 deletions(-) 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 index bc718ee7fa8e6..717898efcb9cd 100644 --- 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 @@ -12,8 +12,8 @@ trait CypherSession { /** * Executes a Cypher query on the given input graph. * - * @param graph [[PropertyGraph]] on which the query is executed - * @param query Cypher query to execute + * @param graph [[PropertyGraph]] on which the query is executed + * @param query Cypher query to execute */ def cypher(graph: PropertyGraph, query: String): CypherResult @@ -34,4 +34,11 @@ trait CypherSession { * @param relationships [[RelationshipFrame]]s that define the relationships in the graph */ def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph + + /** + * Creates a [[PropertyGraph]] from nodes and relationship present in the given [[CypherResult]]. + * + * @param result a [[CypherResult]] containing nodes and/or relationships + */ + def createGraph(result: CypherResult): PropertyGraph } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index ebeac1c20ae4c..b9c6521345dd9 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -10,17 +10,12 @@ import org.opencypher.okapi.ir.api.expr._ import org.opencypher.okapi.relational.api.table.RelationalCypherRecords import org.opencypher.okapi.relational.impl.table.RecordHeader -case class SparkCypherResult( - relationalTable: RelationalCypherRecords[DataFrameTable], - schema: Schema -) extends CypherResult { +case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable], schema: Schema) extends CypherResult { override val df: DataFrame = relationalTable.table.df private val header: RecordHeader = relationalTable.header - private lazy val hasMultipleEntities: Boolean = header.entityVars.size > 1 - // TODO: Error handling override def nodeFrames(varName: String): Seq[NodeFrame] = { val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) @@ -47,10 +42,9 @@ case class SparkCypherResult( .filter(trueLabels.map(df.col).map(_ === true).reduce(_ && _)) .filter(falseLabels.map(df.col).map(_ === false).foldLeft(functions.lit(true))(_ && _)) .select(selectColumns.head, selectColumns.tail: _*) + .dropDuplicates(idColumn) - val distinctDf = if (hasMultipleEntities) labelCombinationDf.dropDuplicates(idColumn) else labelCombinationDf - - NodeFrame(distinctDf, idColumn, labels, properties) + NodeFrame(labelCombinationDf, idColumn, labels, properties) } } @@ -76,10 +70,9 @@ case class SparkCypherResult( val relTypeDf = df .filter(df.col(trueRelType) === true) .select(selectColumns.head, selectColumns.tail: _*) + .dropDuplicates(idColumn) - val distinctDf = if (hasMultipleEntities) relTypeDf.dropDuplicates(idColumn) else relTypeDf - - RelationshipFrame(distinctDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + RelationshipFrame(relTypeDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index c6f92f022a895..d93c4248da8f7 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -5,6 +5,7 @@ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter import org.apache.spark.sql.SparkSession +import org.opencypher.okapi.api.types.{CTNode, CTRelationship} import org.opencypher.okapi.api.value.CypherValue.CypherMap import org.opencypher.okapi.impl.exception.IllegalArgumentException import org.opencypher.okapi.relational.api.graph.{RelationalCypherGraph, RelationalCypherGraphFactory, RelationalCypherSession} @@ -45,6 +46,25 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati RelationalGraphAdapter(this, graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*)) } + override def createGraph(result: CypherResult): PropertyGraph = { + val sparkCypherResult = result match { + case r: SparkCypherResult => r + case other => throw IllegalArgumentException( + expected = "A result that has been created by `SparkCypherSession.cypher`", + actual = other.getClass.getSimpleName + ) + } + + val entityVars = sparkCypherResult.relationalTable.header.entityVars + val nodeVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTNode).isTrue => v.name } + val relVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTRelationship).isTrue => v.name } + + val nodeFrames = nodeVarNames.flatMap(result.nodeFrames).toSeq + val relFrames = relVarNames.flatMap(result.relationshipFrames).toSeq + + createGraph(nodeFrames, relFrames) + } + def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 6d951ae8d02fe..dd141b8dfd8e7 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -48,13 +48,16 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { |RETURN p, o, k |""".stripMargin) + // Option 1: Return NodeFrames and RelationshipFrames val berkeleyStudents: Seq[NodeFrame] = result.nodeFrames("p") val berkeleyStudentFriends: Seq[NodeFrame] = result.nodeFrames("o") val knows: Seq[RelationshipFrame] = result.relationshipFrames("k") - val berkeleyGraph: PropertyGraph = cypherEngine.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() + // Option 2: Use CypherResult to create a new PropertyGraph + val berkeleyGraph2 = cypherEngine.createGraph(result) + berkeleyGraph2.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() } private def id(l: Long): Array[Byte] = BigInt(l).toByteArray From cb3b5b2edc0525d2c74c04d643bcec52d7d7cf62 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 1 Mar 2019 10:47:48 +0100 Subject: [PATCH 027/123] Rename cypherEngine to cypherSession in tests --- .../apache/spark/graph/cypher/BasicMatchSuite.scala | 10 +++++----- .../spark/graph/cypher/SharedCypherContext.scala | 2 +- .../spark/graph/cypher/tck/SparkCypherTckSuite.scala | 8 ++++---- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index dd141b8dfd8e7..9e88b61fc630c 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -10,7 +10,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) - val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame)) graph.cypher("MATCH (n) RETURN n").df.show() } @@ -21,7 +21,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) val relationshipFrame: RelationshipFrame = RelationshipFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val graph: PropertyGraph = cypherEngine.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) graph.nodes.show() graph.relationships.show() @@ -38,7 +38,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val knowsDataFrame: RelationshipFrame = RelationshipFrame(df = knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") val studyAtDataFrame: RelationshipFrame = RelationshipFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") - val graph: PropertyGraph = cypherEngine.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) val result: CypherResult = graph.cypher( """ @@ -52,11 +52,11 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val berkeleyStudents: Seq[NodeFrame] = result.nodeFrames("p") val berkeleyStudentFriends: Seq[NodeFrame] = result.nodeFrames("o") val knows: Seq[RelationshipFrame] = result.relationshipFrames("k") - val berkeleyGraph: PropertyGraph = cypherEngine.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) + val berkeleyGraph: PropertyGraph = cypherSession.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() // Option 2: Use CypherResult to create a new PropertyGraph - val berkeleyGraph2 = cypherEngine.createGraph(result) + val berkeleyGraph2 = cypherSession.createGraph(result) berkeleyGraph2.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() } diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala index c1c64b8026930..b5010761a6dcd 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala @@ -7,7 +7,7 @@ trait SharedCypherContext extends SharedSparkSession { self: Suite => private var _cypherEngine: SparkCypherSession = _ - protected implicit def cypherEngine: SparkCypherSession = _cypherEngine + protected implicit def cypherSession: SparkCypherSession = _cypherEngine override def beforeAll() { super.beforeAll() diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala index c849189c25f2b..dad0288e2d572 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -29,13 +29,13 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { forAll(scenarios.whiteList) { scenario => test(s"[${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { - scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute() + scenario(TCKGraph(graphFactory, cypherSession.graphs.empty)).execute() } } forAll(scenarios.blackList) { scenario => test(s"[${graphFactory.name}, ${BlackList.name}] $scenario", BlackList, TckCapsTag) { - val tckGraph = TCKGraph(graphFactory, cypherEngine.graphs.empty) + val tckGraph = TCKGraph(graphFactory, cypherSession.graphs.empty) Try(scenario(tckGraph).execute()) match { case Success(_) => @@ -97,11 +97,11 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { CypherTCK .parseFilesystemFeature(file) .scenarios - .foreach(scenario => scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute()) + .foreach(scenario => scenario(TCKGraph(graphFactory, cypherSession.graphs.empty)).execute()) } ignore("run single scenario") { scenarios.get("Should add or subtract duration to or from date") - .foreach(scenario => scenario(TCKGraph(graphFactory, cypherEngine.graphs.empty)).execute()) + .foreach(scenario => scenario(TCKGraph(graphFactory, cypherSession.graphs.empty)).execute()) } } From 27abd67773ec96cb428018b9640def64e602ae78 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 10:08:08 +0000 Subject: [PATCH 028/123] Adapt versions to migrate graph work to Spark 3.0 branch MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Sören Reichardt --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 From 402e888439f5c56053d899059441582663b29851 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 10:09:56 +0000 Subject: [PATCH 029/123] Fix to get Spark master branch to compile MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Sören Reichardt --- .../scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 repl/src/main/scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala 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 From 0dddb425c98f027e599c8f510b0f4acac7a7e7af Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 15:07:32 +0000 Subject: [PATCH 030/123] Add methods to read/write property graphs MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Sören Reichardt --- .../spark/graph/api/CypherSession.scala | 24 +++- .../spark/graph/api/PropertyGraph.scala | 7 ++ .../graph/api/io/PropertyGraphReader.scala | 45 ++++++++ .../graph/api/io/PropertyGraphWriter.scala | 61 +++++++++++ .../graph/cypher/SparkCypherSession.scala | 39 +++++++ .../cypher/SparkGraphDirectoryStructure.scala | 65 +++++++++++ .../conversions/StringEncodingUtilities.scala | 103 ++++++++++++++++++ .../graph/cypher/io/ReadWriteGraph.scala | 102 +++++++++++++++++ .../graph/cypher/util/HadoopFSUtils.scala | 24 ++++ .../graph/cypher/PropertyGraphReadWrite.scala | 45 ++++++++ 10 files changed, 514 insertions(+), 1 deletion(-) create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/StringEncodingUtilities.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala create mode 100644 graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala 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 index 717898efcb9cd..2505b5e948d23 100644 --- 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 @@ -1,6 +1,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.SparkSession +import org.apache.spark.graph.api.io.{PropertyGraphReader, PropertyGraphWriter, ReaderConfig, WriterConfig} +import org.apache.spark.sql.{SaveMode, SparkSession} /** * Allows for creating [[PropertyGraph]] instances and running Cypher-queries on them. @@ -41,4 +42,25 @@ trait CypherSession { * @param result a [[CypherResult]] containing nodes and/or relationships */ def createGraph(result: CypherResult): PropertyGraph + + /** + * Returns a [[PropertyGraphWriter]] for `graph`. + */ + def write(graph: PropertyGraph): PropertyGraphWriter = PropertyGraphWriter(graph, WriterConfig( + path = ".", + SaveMode.ErrorIfExists, + sparkSession.sessionState.conf.defaultDataSourceName) + ) + + /** + * Returns a [[PropertyGraphReader]]. + */ + def read: PropertyGraphReader = PropertyGraphReader(this, ReaderConfig( + path = ".", + sparkSession.sessionState.conf.defaultDataSourceName)) + + private[spark] def readGraph(config: ReaderConfig): PropertyGraph + + private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit + } 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 index 3ae8056cf99c4..ebd98330172f1 100644 --- 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 @@ -1,5 +1,6 @@ package org.apache.spark.graph.api +import org.apache.spark.graph.api.io.PropertyGraphWriter import org.apache.spark.sql.DataFrame /** @@ -33,4 +34,10 @@ trait PropertyGraph { * Returns a [[DataFrame]] that contains a row for each relationship in this graph. */ def relationships: DataFrame + + /** + * Returns a [[PropertyGraphWriter]]. + */ + def write: PropertyGraphWriter = cypherSession.write(this) + } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala new file mode 100644 index 0000000000000..cffa149bc33a0 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala @@ -0,0 +1,45 @@ +package org.apache.spark.graph.api.io + +import org.apache.spark.graph.api.{CypherSession, PropertyGraph} + +private[spark] case class ReaderConfig( + path: String, + source: String +) + +case class PropertyGraphReader( + session: CypherSession, + config: ReaderConfig +) { + + /** + * Saves the content of the `PropertyGraph` at the specified path. + */ + def read(path: String): PropertyGraph = { + session.readGraph(config.copy(path = path)) + } + + /** + * Saves the content of the `DataFrame` in Parquet format at the specified path. + */ + def parquet(path: String): PropertyGraph = { + format("parquet").read(path) + } + + /** + * Saves the content of the `DataFrame` in ORC format at the specified path. + * + * @note Currently, this method can only be used after enabling Hive support + */ + def orc(path: String): PropertyGraph = { + format("orc").read(path) + } + + /** + * Specifies the underlying output data source. Built-in options include "parquet", "json", etc. + */ + private[spark] def format(source: String): PropertyGraphReader = { + copy(config = config.copy(source = source)) + } + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala new file mode 100644 index 0000000000000..03eec798dcb4f --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala @@ -0,0 +1,61 @@ +package org.apache.spark.graph.api.io + +import org.apache.spark.graph.api.PropertyGraph +import org.apache.spark.sql.SaveMode + +private[spark] case class WriterConfig( + path: String, + mode: SaveMode, + source: String +) + +case class PropertyGraphWriter( + graph: PropertyGraph, + config: WriterConfig +) { + + /** + * Saves the content of the `PropertyGraph` at the specified path. + */ + def save(path: String): Unit = { + copy(config = config.copy(path = path)).save() + } + + /** + * Specifies the behavior when data or table already exists. Options include: + *
    + *
  • `SaveMode.Overwrite`: overwrite the existing data.
  • + *
  • `SaveMode.Append`: append the data.
  • + *
  • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
  • + *
  • `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
  • + *
+ */ + def mode(saveMode: SaveMode): PropertyGraphWriter = { + copy(config = config.copy(mode = saveMode)) + } + + private[spark] def format(source: String): PropertyGraphWriter = { + copy(config = config.copy(source = source)) + } + + /** + * Saves the content of the `PropertyGraph` in Parquet format at the specified path. + */ + def parquet(path: String): Unit = { + format("parquet").save(path) + } + + /** + * Saves the content of the `DataFrame` in ORC format at the specified path. + * + * @note Currently, this method can only be used after enabling Hive support + */ + def orc(path: String): Unit = { + format("orc").save(path) + } + + private[spark] def save(): Unit = { + graph.cypherSession.writeGraph(graph, config) + } + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index d93c4248da8f7..80ed53b5152c5 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -1,9 +1,11 @@ package org.apache.spark.graph.cypher import org.apache.spark.graph.api._ +import org.apache.spark.graph.api.io.{ReaderConfig, WriterConfig} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter +import org.apache.spark.graph.cypher.io.ReadWriteGraph.GraphImporter import org.apache.spark.sql.SparkSession import org.opencypher.okapi.api.types.{CTNode, CTRelationship} import org.opencypher.okapi.api.value.CypherValue.CypherMap @@ -78,4 +80,41 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema) } + + override private[spark] def readGraph(config: ReaderConfig): PropertyGraph = { + val graphImporter = GraphImporter(sparkSession, config) + try { + createGraph(graphImporter.nodeFrames, graphImporter.relationshipFrames) + } finally { + graphImporter.close() + } + } + + override private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit = { + import org.apache.spark.graph.cypher.io.ReadWriteGraph._ + + val relationalGraph = graph match { + case RelationalGraphAdapter(_, relGraph) => relGraph + case other => throw IllegalArgumentException( + expected = "A graph that has been created by `SparkCypherSession.createGraph`", + actual = other.getClass.getSimpleName + ) + } + + val graphDirectoryStructure = SparkGraphDirectoryStructure(config.path) + + relationalGraph.schema.labelCombinations.combos.foreach { combo => + relationalGraph.canonicalNodeTable(combo).write + .format(config.source) + .mode(config.mode) + .save(graphDirectoryStructure.pathToNodeTable(combo)) + } + relationalGraph.schema.relationshipTypes.foreach { relType => + relationalGraph.canonicalRelationshipTable(relType).write + .format(config.source) + .mode(config.mode) + .save(graphDirectoryStructure.pathToRelationshipTable(relType)) + } + } + } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala new file mode 100644 index 0000000000000..814a452763e1d --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala @@ -0,0 +1,65 @@ +package org.apache.spark.graph.cypher + +import org.apache.hadoop.fs.Path + +object SparkGraphDirectoryStructure { + + import org.apache.spark.graph.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/graph/cypher/conversions/StringEncodingUtilities.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/StringEncodingUtilities.scala new file mode 100644 index 0000000000000..ced0464448fba --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/StringEncodingUtilities.scala @@ -0,0 +1,103 @@ +package org.apache.spark.graph.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/graph/cypher/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala new file mode 100644 index 0000000000000..d1c4959e9f2e6 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala @@ -0,0 +1,102 @@ +package org.apache.spark.graph.cypher.io + +import java.net.URI + +import org.apache.hadoop.fs.FileSystem +import org.apache.spark.graph.api.io.ReaderConfig +import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} +import org.apache.spark.graph.cypher.SparkGraphDirectoryStructure +import org.apache.spark.graph.cypher.SparkGraphDirectoryStructure._ +import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.graph.cypher.conversions.StringEncodingUtilities._ +import org.apache.spark.sql.{DataFrame, 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, config: ReaderConfig) { + import org.apache.spark.graph.cypher.util.HadoopFSUtils._ + + val directoryStructure: SparkGraphDirectoryStructure = SparkGraphDirectoryStructure(config.path) + val fs: FileSystem = FileSystem.get(new URI(config.path), sparkSession.sparkContext.hadoopConfiguration) + + def nodeFrames: Seq[NodeFrame] = { + val nodeLabelComboDirectories = fs.listDirectories(directoryStructure.pathToNodeDirectory) + val labelCombos = nodeLabelComboDirectories.map(_.toLabelCombo) + labelCombos.map { combo => + val df = sparkSession.read.format(config.source).load(directoryStructure.pathToNodeTable(combo)) + val propertyMappings = df.columns.filter(_.isPropertyColumnName).map(colName => colName.toProperty -> colName).toMap + NodeFrame( + df, + SourceIdKey.name, + combo, + propertyMappings) + } + } + + def relationshipFrames: Seq[RelationshipFrame] = { + val relTypeDirectories = fs.listDirectories(directoryStructure.pathToRelationshipDirectory) + val relTypes = relTypeDirectories.map(_.toRelationshipType) + relTypes.map { relType => + val df = sparkSession.read.format(config.source).load(directoryStructure.pathToRelationshipTable(relType)) + val propertyMappings = df.columns.filter(_.isPropertyColumnName).map(colName => colName.toProperty -> colName).toMap + RelationshipFrame( + df, + SourceIdKey.name, + SourceStartNodeKey.name, + SourceEndNodeKey.name, + relType, + propertyMappings) + } + } + + def close(): Unit = { + fs.close() + } + + } + + implicit class GraphExport(graph: RelationalCypherGraph[DataFrameTable]) { + + def canonicalNodeTable(labels: Set[String]): DataFrame = { + 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): DataFrame = { + 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/graph/cypher/util/HadoopFSUtils.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala new file mode 100644 index 0000000000000..9aec11715021a --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala @@ -0,0 +1,24 @@ +package org.apache.spark.graph.cypher.util + +import org.apache.hadoop.fs.{FileSystem, Path} + +import scala.util.Try + +object HadoopFSUtils { + + implicit class RichHadoopFileSystem(fileSystem: FileSystem) { + + def listDirectories(path: String): List[String] = { + val p = new Path(path) + Try( + fileSystem + .listStatus(p) + .filter(_.isDirectory) + .map(_.getPath.getName) + .toList + ).getOrElse(List.empty) + } + + } + +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala new file mode 100644 index 0000000000000..fc15839a81e1c --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala @@ -0,0 +1,45 @@ +package org.apache.spark.graph.cypher + +import java.nio.file.Paths + +import org.apache.spark.SparkFunSuite +import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} +import org.apache.spark.sql.DataFrame +import org.junit.rules.TemporaryFolder +import org.scalatest.BeforeAndAfterEach + +class PropertyGraphReadWrite extends SparkFunSuite 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: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") + private lazy val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") + private lazy val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) + private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + + test("write a graph with orc") { + cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)).write.orc(basePath) + cypherSession.read.orc(basePath).cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() + } + + test("write a graph with parquet") { + cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)).write.parquet(basePath) + cypherSession.read.parquet(basePath).cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() + } + + private def id(l: Long): Array[Byte] = BigInt(l).toByteArray + +} From 763e99cbd65c0be13d3342dc3d7f827e89dcace8 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 15:24:20 +0000 Subject: [PATCH 031/123] Use collect instead of filter + map --- .../apache/spark/graph/cypher/io/ReadWriteGraph.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala index d1c4959e9f2e6..dd8ce5fc41072 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala @@ -18,6 +18,7 @@ import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph object ReadWriteGraph { case class GraphImporter(sparkSession: SparkSession, config: ReaderConfig) { + import org.apache.spark.graph.cypher.util.HadoopFSUtils._ val directoryStructure: SparkGraphDirectoryStructure = SparkGraphDirectoryStructure(config.path) @@ -28,7 +29,9 @@ object ReadWriteGraph { val labelCombos = nodeLabelComboDirectories.map(_.toLabelCombo) labelCombos.map { combo => val df = sparkSession.read.format(config.source).load(directoryStructure.pathToNodeTable(combo)) - val propertyMappings = df.columns.filter(_.isPropertyColumnName).map(colName => colName.toProperty -> colName).toMap + val propertyMappings = df.columns.collect { + case colName if colName.isPropertyColumnName => colName.toProperty -> colName + }.toMap NodeFrame( df, SourceIdKey.name, @@ -42,7 +45,9 @@ object ReadWriteGraph { val relTypes = relTypeDirectories.map(_.toRelationshipType) relTypes.map { relType => val df = sparkSession.read.format(config.source).load(directoryStructure.pathToRelationshipTable(relType)) - val propertyMappings = df.columns.filter(_.isPropertyColumnName).map(colName => colName.toProperty -> colName).toMap + val propertyMappings = df.columns.collect { + case colName if colName.isPropertyColumnName => colName.toProperty -> colName + }.toMap RelationshipFrame( df, SourceIdKey.name, From cd2f7305f2c80145e0ee24dcbe5e52dba1b6cb16 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 15:35:02 +0000 Subject: [PATCH 032/123] Simplify filesystem resource handling --- .../graph/cypher/SparkCypherSession.scala | 6 +---- .../graph/cypher/io/ReadWriteGraph.scala | 22 +++++++++---------- .../graph/cypher/util/HadoopFSUtils.scala | 15 ++++++------- 3 files changed, 19 insertions(+), 24 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 80ed53b5152c5..e7eaf4ae7be83 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -83,11 +83,7 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati override private[spark] def readGraph(config: ReaderConfig): PropertyGraph = { val graphImporter = GraphImporter(sparkSession, config) - try { - createGraph(graphImporter.nodeFrames, graphImporter.relationshipFrames) - } finally { - graphImporter.close() - } + createGraph(graphImporter.nodeFrames, graphImporter.relationshipFrames) } override private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit = { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala index dd8ce5fc41072..4cfa29b863718 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala @@ -19,14 +19,20 @@ object ReadWriteGraph { case class GraphImporter(sparkSession: SparkSession, config: ReaderConfig) { - import org.apache.spark.graph.cypher.util.HadoopFSUtils._ - val directoryStructure: SparkGraphDirectoryStructure = SparkGraphDirectoryStructure(config.path) - val fs: FileSystem = FileSystem.get(new URI(config.path), sparkSession.sparkContext.hadoopConfiguration) + val (labelCombos, relTypes): (Seq[Set[String]], Seq[String]) = { + val fs = FileSystem.get(new URI(config.path), sparkSession.sparkContext.hadoopConfiguration) + try { + import org.apache.spark.graph.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 nodeFrames: Seq[NodeFrame] = { - val nodeLabelComboDirectories = fs.listDirectories(directoryStructure.pathToNodeDirectory) - val labelCombos = nodeLabelComboDirectories.map(_.toLabelCombo) labelCombos.map { combo => val df = sparkSession.read.format(config.source).load(directoryStructure.pathToNodeTable(combo)) val propertyMappings = df.columns.collect { @@ -41,8 +47,6 @@ object ReadWriteGraph { } def relationshipFrames: Seq[RelationshipFrame] = { - val relTypeDirectories = fs.listDirectories(directoryStructure.pathToRelationshipDirectory) - val relTypes = relTypeDirectories.map(_.toRelationshipType) relTypes.map { relType => val df = sparkSession.read.format(config.source).load(directoryStructure.pathToRelationshipTable(relType)) val propertyMappings = df.columns.collect { @@ -58,10 +62,6 @@ object ReadWriteGraph { } } - def close(): Unit = { - fs.close() - } - } implicit class GraphExport(graph: RelationalCypherGraph[DataFrameTable]) { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala index 9aec11715021a..35a72190c3340 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala @@ -8,15 +8,14 @@ object HadoopFSUtils { implicit class RichHadoopFileSystem(fileSystem: FileSystem) { - def listDirectories(path: String): List[String] = { + def listDirectories(path: String): Seq[String] = { val p = new Path(path) - Try( - fileSystem - .listStatus(p) - .filter(_.isDirectory) - .map(_.getPath.getName) - .toList - ).getOrElse(List.empty) + fileSystem + .listStatus(p) + .collect { case item if item.isDirectory => + item.getPath.getName + } + .toSeq } } From 1884a7d9b9517a895cc42b62f190bbbb9c1540aa Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 15:38:37 +0000 Subject: [PATCH 033/123] Improve code reuse in SparkCypherSession --- .../graph/cypher/SparkCypherSession.scala | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index e7eaf4ae7be83..550abb283983b 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -70,14 +70,7 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { - val relationalGraph = graph match { - case RelationalGraphAdapter(_, relGraph) => relGraph - case other => throw IllegalArgumentException( - expected = "A graph that has been created by `SparkCypherSession.createGraph`", - actual = other.getClass.getSimpleName - ) - } - + val relationalGraph = toRelationalGraph(graph) SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema) } @@ -88,15 +81,7 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati override private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit = { import org.apache.spark.graph.cypher.io.ReadWriteGraph._ - - val relationalGraph = graph match { - case RelationalGraphAdapter(_, relGraph) => relGraph - case other => throw IllegalArgumentException( - expected = "A graph that has been created by `SparkCypherSession.createGraph`", - actual = other.getClass.getSimpleName - ) - } - + val relationalGraph = toRelationalGraph(graph) val graphDirectoryStructure = SparkGraphDirectoryStructure(config.path) relationalGraph.schema.labelCombinations.combos.foreach { combo => @@ -113,4 +98,14 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati } } + private def toRelationalGraph(graph: PropertyGraph): RelationalCypherGraph[DataFrameTable] = { + graph match { + case RelationalGraphAdapter(_, relGraph) => relGraph + case other => throw IllegalArgumentException( + expected = "A graph that has been created by `SparkCypherSession.createGraph`", + actual = other.getClass.getSimpleName + ) + } + } + } From 1bfb5405f8ecc899e7785abb5640fb72abc411dc Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 16:03:17 +0000 Subject: [PATCH 034/123] Throw `UnsupportedOperationException` when `entityTables` is called --- .../graph/cypher/SparkCypherSession.scala | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 550abb283983b..fc5da1b015244 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -5,11 +5,11 @@ import org.apache.spark.graph.api.io.{ReaderConfig, WriterConfig} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter -import org.apache.spark.graph.cypher.io.ReadWriteGraph.GraphImporter +import org.apache.spark.graph.cypher.io.ReadWriteGraph._ import org.apache.spark.sql.SparkSession import org.opencypher.okapi.api.types.{CTNode, CTRelationship} import org.opencypher.okapi.api.value.CypherValue.CypherMap -import org.opencypher.okapi.impl.exception.IllegalArgumentException +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.RelationalEntityTableFactory @@ -34,11 +34,15 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati override val records: SparkCypherRecordsFactory = SparkCypherRecordsFactory() - override val graphs: RelationalCypherGraphFactory[DataFrameTable] = new RelationalCypherGraphFactory[DataFrameTable]() { - override implicit val session: RelationalCypherSession[DataFrameTable] = sparkCypherSession + override val graphs: RelationalCypherGraphFactory[DataFrameTable] = { + new RelationalCypherGraphFactory[DataFrameTable]() { + override implicit val session: RelationalCypherSession[DataFrameTable] = sparkCypherSession + } } - override def entityTables: RelationalEntityTableFactory[DataFrameTable] = ??? + override def entityTables: RelationalEntityTableFactory[DataFrameTable] = { + throw UnsupportedOperationException("Graph construction with `CONSTRUCT` is not supported in Cypher 9") + } override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") @@ -80,18 +84,19 @@ class SparkCypherSession(override val sparkSession: SparkSession) extends Relati } override private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit = { - import org.apache.spark.graph.cypher.io.ReadWriteGraph._ val relationalGraph = toRelationalGraph(graph) val graphDirectoryStructure = SparkGraphDirectoryStructure(config.path) relationalGraph.schema.labelCombinations.combos.foreach { combo => - relationalGraph.canonicalNodeTable(combo).write + relationalGraph.canonicalNodeTable(combo) + .write .format(config.source) .mode(config.mode) .save(graphDirectoryStructure.pathToNodeTable(combo)) } relationalGraph.schema.relationshipTypes.foreach { relType => - relationalGraph.canonicalRelationshipTable(relType).write + relationalGraph.canonicalRelationshipTable(relType) + .write .format(config.source) .mode(config.mode) .save(graphDirectoryStructure.pathToRelationshipTable(relType)) From 42f7896199ca9fabb5e70dd8bf701600ea5faafd Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Mon, 4 Mar 2019 16:15:40 +0000 Subject: [PATCH 035/123] Hide internals of `SparkCypherSession` outside of `spark` package --- .../org/apache/spark/graph/api/CypherSession.scala | 2 +- .../apache/spark/graph/cypher/SparkCypherSession.scala | 6 ++++-- .../spark/graph/cypher/SharedCypherContext.scala | 7 +++++-- .../spark/graph/cypher/tck/SparkCypherTckSuite.scala | 10 +++++----- 4 files changed, 15 insertions(+), 10 deletions(-) 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 index 2505b5e948d23..6a90d874254bb 100644 --- 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 @@ -34,7 +34,7 @@ trait CypherSession { * @param nodes [[NodeFrame]]s that define the nodes in the graph * @param relationships [[RelationshipFrame]]s that define the relationships in the graph */ - def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph + def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph /** * Creates a [[PropertyGraph]] from nodes and relationship present in the given [[CypherResult]]. diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index fc5da1b015244..74cf8fcad0729 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -15,7 +15,9 @@ import org.opencypher.okapi.relational.api.planning.RelationalCypherResult import org.opencypher.okapi.relational.api.table.RelationalEntityTableFactory object SparkCypherSession { - def create(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) + def create(implicit sparkSession: SparkSession): CypherSession = new SparkCypherSession(sparkSession) + + private[spark] def createInternal(implicit sparkSession: SparkSession): SparkCypherSession = new SparkCypherSession(sparkSession) } /** @@ -24,7 +26,7 @@ object SparkCypherSession { * This class is the main entry point for working with the spark-graph-cypher module. * It wraps a [[SparkSession]] and allows to run Cypher queries over graphs represented as [[org.apache.spark.sql.DataFrame]]s. */ -class SparkCypherSession(override val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherSession { +private[spark] class SparkCypherSession(override val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherSession { override type Result = RelationalCypherResult[DataFrameTable] override type Records = SparkCypherRecords diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala index b5010761a6dcd..9db9f8c1c4983 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala @@ -1,5 +1,6 @@ package org.apache.spark.graph.cypher +import org.apache.spark.graph.api.CypherSession import org.apache.spark.sql.test.SharedSparkSession import org.scalatest.Suite @@ -7,11 +8,13 @@ trait SharedCypherContext extends SharedSparkSession { self: Suite => private var _cypherEngine: SparkCypherSession = _ - protected implicit def cypherSession: SparkCypherSession = _cypherEngine + protected implicit def cypherSession: CypherSession = _cypherEngine + + def internalCypherSession: SparkCypherSession = _cypherEngine override def beforeAll() { super.beforeAll() - _cypherEngine = SparkCypherSession.create + _cypherEngine = SparkCypherSession.createInternal } protected override def afterAll(): Unit = { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala index dad0288e2d572..a6878dad94683 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -3,8 +3,8 @@ package org.apache.spark.graph.cypher.tck import java.io.File import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.cypher.{SharedCypherContext, SparkCypherSession} import org.apache.spark.graph.cypher.construction.ScanGraphFactory +import org.apache.spark.graph.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 @@ -29,13 +29,13 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { forAll(scenarios.whiteList) { scenario => test(s"[${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { - scenario(TCKGraph(graphFactory, cypherSession.graphs.empty)).execute() + scenario(TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession)).execute() } } forAll(scenarios.blackList) { scenario => test(s"[${graphFactory.name}, ${BlackList.name}] $scenario", BlackList, TckCapsTag) { - val tckGraph = TCKGraph(graphFactory, cypherSession.graphs.empty) + val tckGraph = TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession) Try(scenario(tckGraph).execute()) match { case Success(_) => @@ -97,11 +97,11 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { CypherTCK .parseFilesystemFeature(file) .scenarios - .foreach(scenario => scenario(TCKGraph(graphFactory, cypherSession.graphs.empty)).execute()) + .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, cypherSession.graphs.empty)).execute()) + .foreach(scenario => scenario(TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession)).execute()) } } From d7fa40e113ce4a00cd2a7c63663eaedaeab71752 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Tue, 5 Mar 2019 11:00:22 +0000 Subject: [PATCH 036/123] Automatically convert ID columns that don't have BinaryType MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Sören Reichardt --- .../spark/graph/api/GraphElementFrame.scala | 74 ++++++++++++++++--- .../spark/graph/cypher/BasicMatchSuite.scala | 23 +++--- 2 files changed, 73 insertions(+), 24 deletions(-) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 2b28005c6ff47..e40c1de83bc0b 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -1,16 +1,55 @@ package org.apache.spark.graph.api +import org.apache.spark.graph.api.GraphElementFrame.encodeIdColumns import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{BinaryType, StringType} + +object GraphElementFrame { + + // TODO: Consider creating an expression for this + def encodeIdColumns(df: DataFrame, idColumnNames: String*): DataFrame = { + val encodedIdCols = idColumnNames.map { idColumnName => + val col = df.col(idColumnName) + df.schema(idColumnName).dataType match { + case BinaryType => col + case StringType => col.cast(BinaryType) + // TODO: Implement more efficient encoding for IntegerType and LongType + // TODO: Constraint to types that make sense as IDs + case _ => col.cast(StringType).cast(BinaryType) + } + } + val remainingColumnNames = df.columns.filterNot(idColumnNames.contains) + val remainingCols = remainingColumnNames.map(df.col) + df.select(encodedIdCols ++ remainingCols: _*) + } + +} /** * Describes how to map an input [[DataFrame]] to graph elements (i.e. nodes or relationships). */ trait GraphElementFrame { + def initialDf: DataFrame + /** - * [[DataFrame]] containing element data. Each row represents a graph element. + * [[DataFrame]] containing only mapped element data. Each row represents a graph element. + * Columns in 'initialDf' that do not have BinaryType are converted to BinaryType. + * Id columns are first, property columns are sorted alphabetically. */ - def df: DataFrame + val df: DataFrame = { + val mappedColumnNames = idColumns ++ properties.values.toSeq.sorted + val mappedDf = if (mappedColumnNames == initialDf.columns.toSeq) { + initialDf + } else { + initialDf.select(mappedColumnNames.map(initialDf.col): _*) + } + if (idColumns.forall(idColumn => initialDf.schema(idColumn).dataType == BinaryType)) { + mappedDf + } else { + encodeIdColumns(mappedDf, idColumns: _*) + } + } /** * Name of the column that contains the graph element identifier. @@ -23,6 +62,9 @@ trait GraphElementFrame { * Mapping from graph element property keys to the columns that contain the corresponding property values. */ def properties: Map[String, String] + + protected def idColumns: Seq[String] + } object NodeFrame { @@ -51,17 +93,21 @@ object NodeFrame { /** * Describes how to map an input [[DataFrame]] to nodes. * - * @param df [[DataFrame]] containing a single node in each row + * @param initialDf [[DataFrame]] containing a single node in each row * @param idColumn column that contains the node identifier * @param labels labels that are assigned to all nodes * @param properties mapping from property keys to corresponding columns */ case class NodeFrame( - df: DataFrame, + initialDf: DataFrame, idColumn: String, labels: Set[String], properties: Map[String, String] -) extends GraphElementFrame +) extends GraphElementFrame { + + override protected def idColumns: Seq[String] = Seq(idColumn) + +} object RelationshipFrame { @@ -70,22 +116,22 @@ object RelationshipFrame { * * All columns apart from the given identifier columns are mapped to relationship properties. * - * @param df [[DataFrame]] containing a single relationship in each row + * @param initialDf [[DataFrame]] 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 */ def apply( - df: DataFrame, + initialDf: DataFrame, idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String ): RelationshipFrame = { - val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) + val properties = (initialDf.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) .map(columnName => columnName -> columnName).toMap - RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + RelationshipFrame(initialDf, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) } } @@ -93,7 +139,7 @@ object RelationshipFrame { /** * Describes how to map an input [[DataFrame]] to relationships. * - * @param df [[DataFrame]] containing a single relationship in each row + * @param initialDf [[DataFrame]] 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 @@ -101,10 +147,14 @@ object RelationshipFrame { * @param properties mapping from property keys to corresponding columns */ case class RelationshipFrame( - df: DataFrame, + initialDf: DataFrame, idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String, properties: Map[String, String] -) extends GraphElementFrame +) extends GraphElementFrame { + + override protected def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) + +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 9e88b61fc630c..95f524aa1f152 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -7,7 +7,7 @@ import org.apache.spark.sql.DataFrame class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("match single node pattern using spark-graph-api") { - val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") + val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame)) @@ -16,10 +16,10 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { } test("match simple pattern using spark-graph-api") { - val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") - val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") + val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") + val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, 0, 1))).toDF("id", "source", "target") val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) - val relationshipFrame: RelationshipFrame = RelationshipFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) graph.nodes.show() @@ -29,14 +29,14 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { } test("create property graph from query results") { - val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), "Alice", 42), Tuple3(id(1), "Bob", 23), Tuple3(id(2), "Eve", 19))).toDF("id", "name", "age") - val universityData: DataFrame = spark.createDataFrame(Seq(id(2) -> "UC Berkeley", id(3)-> "Stanford")).toDF("id", "title") - val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)), Tuple3(id(1), id(0), id(2)))).toDF("id", "source", "target") - val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(2), id(0), id(2)), Tuple3(id(3), id(1), id(3)), Tuple3(id(4), id(2), id(2)))).toDF("id", "source", "target") + val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, "Alice", 42), Tuple3(1, "Bob", 23), Tuple3(2, "Eve", 19))).toDF("id", "name", "age") + val universityData: DataFrame = spark.createDataFrame(Seq(2 -> "UC Berkeley", 3-> "Stanford")).toDF("id", "title") + val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, 0, 1), Tuple3(1, 0, 2))).toDF("id", "source", "target") + val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(2, 0, 2), Tuple3(3, 1, 3), Tuple3(4, 2, 2))).toDF("id", "source", "target") val personDataFrame: NodeFrame = NodeFrame(df = personData, idColumn = "id", labels = Set("Student")) val universityDataFrame: NodeFrame = NodeFrame(df = universityData, idColumn = "id", labels = Set("University")) - val knowsDataFrame: RelationshipFrame = RelationshipFrame(df = knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val studyAtDataFrame: RelationshipFrame = RelationshipFrame(df = studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") + val knowsDataFrame: RelationshipFrame = RelationshipFrame(knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val studyAtDataFrame: RelationshipFrame = RelationshipFrame(studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") val graph: PropertyGraph = cypherSession.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) @@ -59,6 +59,5 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val berkeleyGraph2 = cypherSession.createGraph(result) berkeleyGraph2.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() } - - private def id(l: Long): Array[Byte] = BigInt(l).toByteArray + } From af840de205351d9b14302a265da85045bc5f58e2 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Tue, 5 Mar 2019 11:13:03 +0000 Subject: [PATCH 037/123] Improve documentation --- .../org/apache/spark/graph/api/GraphElementFrame.scala | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index e40c1de83bc0b..b52f51aeb4b0f 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -14,7 +14,7 @@ object GraphElementFrame { case BinaryType => col case StringType => col.cast(BinaryType) // TODO: Implement more efficient encoding for IntegerType and LongType - // TODO: Constraint to types that make sense as IDs + // TODO: Constrain to types that make sense as IDs case _ => col.cast(StringType).cast(BinaryType) } } @@ -30,12 +30,16 @@ object GraphElementFrame { */ trait GraphElementFrame { + /** + * Initial [[DataFrame]] that can still contain unmapped columns and ID columns that are not of type BinaryType. + * The columns are ordered arbitrarily. + */ def initialDf: DataFrame /** - * [[DataFrame]] containing only mapped element data. Each row represents a graph element. + * [[DataFrame]] that contains only mapped element data. Each row represents a graph element. * Columns in 'initialDf' that do not have BinaryType are converted to BinaryType. - * Id columns are first, property columns are sorted alphabetically. + * ID columns are first, property columns are sorted alphabetically. */ val df: DataFrame = { val mappedColumnNames = idColumns ++ properties.values.toSeq.sorted From 77b7be084ce46cceb2342f94e5e153aac2a68826 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Wed, 6 Mar 2019 13:49:44 +0000 Subject: [PATCH 038/123] Remove workaround for `distinct` bug MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Tobias Johansson Co-authored-by: Sören Reichardt --- .../spark/graph/cypher/SparkTable.scala | 19 +------------------ .../graph/cypher/PropertyGraphReadWrite.scala | 2 +- 2 files changed, 2 insertions(+), 19 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index 32a5bd30e240a..161dd66860ad9 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -200,25 +200,8 @@ object SparkTable { override def distinct: DataFrameTable = distinct(df.columns: _*) - // workaround for https://issues.apache.org/jira/browse/SPARK-26572 override def distinct(colNames: String*): DataFrameTable = { - val uniqueSuffix = "_temp_distinct" - - val originalColNames = df.columns - - val renamings = originalColNames.map { c => - if (colNames.contains(c)) c -> s"$c$uniqueSuffix" - else c -> c - }.toMap - - val renamedDf = df.safeRenameColumns(colNames.map(c => c -> renamings(c)).toMap) - - val extractRowFromGrouping = originalColNames.map(c => functions.first(renamings(c)) as c) - val groupedDf = renamedDf - .groupBy(colNames.map(c => functions.col(renamings(c))): _*) - .agg(extractRowFromGrouping.head, extractRowFromGrouping.tail: _*) - - groupedDf.safeDropColumns(colNames.map(renamings): _*) + df.dropDuplicates(colNames) } override def withColumnsRenamed(columnRenamings: Map[String, String]): DataFrameTable = { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala index fc15839a81e1c..bbebadc8350f0 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala @@ -28,7 +28,7 @@ class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with private lazy val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") private lazy val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") private lazy val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) - private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame(df = relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") test("write a graph with orc") { cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)).write.orc(basePath) From b56342310b94f7055497a65a3c6ed1a9e7d9f554 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 5 Mar 2019 18:11:17 +0100 Subject: [PATCH 039/123] Extend Cast expression with support for casting integer types to binary Co-authored-by: Philip Stutz --- .../org/apache/spark/sql/catalyst/CatalystTypeConverters.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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, From 4ed8bf83b9cd19641313c8eb5dbe7743f40639dc Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Wed, 6 Mar 2019 13:55:52 +0000 Subject: [PATCH 040/123] Use new cast(BinaryType) feature for Byte|Short|Integer|Long conversion MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Tobias Johansson Co-authored-by: Sören Reichardt --- .../org/apache/spark/graph/api/GraphElementFrame.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index b52f51aeb4b0f..6b12bdd9193cd 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -2,18 +2,16 @@ package org.apache.spark.graph.api import org.apache.spark.graph.api.GraphElementFrame.encodeIdColumns import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.types.{BinaryType, StringType} +import org.apache.spark.sql.types._ object GraphElementFrame { - // TODO: Consider creating an expression for this def encodeIdColumns(df: DataFrame, idColumnNames: String*): DataFrame = { val encodedIdCols = idColumnNames.map { idColumnName => val col = df.col(idColumnName) df.schema(idColumnName).dataType match { case BinaryType => col - case StringType => col.cast(BinaryType) - // TODO: Implement more efficient encoding for IntegerType and LongType + case StringType | ByteType | ShortType | IntegerType | LongType => col.cast(BinaryType) // TODO: Constrain to types that make sense as IDs case _ => col.cast(StringType).cast(BinaryType) } From 0b4c90aa1531f1211986563930234ea95165c73c Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Fri, 8 Mar 2019 08:36:24 +0000 Subject: [PATCH 041/123] Remove id function in test --- .../graph/cypher/PropertyGraphReadWrite.scala | 40 ++++++++++++++----- 1 file changed, 30 insertions(+), 10 deletions(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala index bbebadc8350f0..cccc306cb97b8 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala @@ -25,21 +25,41 @@ class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with private def basePath: String = s"file://${Paths.get(tempDir.getRoot.getAbsolutePath)}" - private lazy val nodeData: DataFrame = spark.createDataFrame(Seq(id(0) -> "Alice", id(1) -> "Bob")).toDF("id", "name") - private lazy val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(id(0), id(0), id(1)))).toDF("id", "source", "target") - private lazy val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) - private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + private lazy val nodeData: DataFrame = spark.createDataFrame(Seq( + 0 -> "Alice", + 1 -> "Bob" + )).toDF("id", "name") + + private lazy val relationshipData: DataFrame = spark.createDataFrame(Seq( + Tuple3(0, 0, 1) + )).toDF("id", "source", "target") + + private lazy val nodeDataFrame: NodeFrame = NodeFrame( + df = nodeData, idColumn = "id", labels = Set("Person") + ) + + private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame( + relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS" + ) test("write a graph with orc") { - cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)).write.orc(basePath) - cypherSession.read.orc(basePath).cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() + val graph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) + graph.write.orc(basePath) + + val readGraph = cypherSession.read.orc(basePath) + readGraph.cypher( + "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" + ).df.show() } test("write a graph with parquet") { - cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)).write.parquet(basePath) - cypherSession.read.parquet(basePath).cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() - } + val graph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) + graph.write.parquet(basePath) - private def id(l: Long): Array[Byte] = BigInt(l).toByteArray + val readGraph = cypherSession.read.parquet(basePath) + readGraph.cypher( + "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" + ).df.show() + } } From 46537bc1e9dfe518d64343570a56106c7622fd1c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Max=20Kie=C3=9Fling?= Date: Tue, 12 Mar 2019 18:11:11 +0100 Subject: [PATCH 042/123] Update to latest OKAPI release Co-authored-by: Martin Junghanns --- .../graph/cypher/SparkCypherSession.scala | 4 +- .../spark/graph/cypher/SparkTable.scala | 4 - .../cypher/conversions/ExprConversions.scala | 173 +++++++++--------- .../src/test/resources/tck/failing_blacklist | 32 ++-- .../src/test/resources/tck/temporal_blacklist | 24 ++- 5 files changed, 132 insertions(+), 105 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 74cf8fcad0729..f51e87aaf0b4b 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -64,8 +64,8 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) } val entityVars = sparkCypherResult.relationalTable.header.entityVars - val nodeVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTNode).isTrue => v.name } - val relVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTRelationship).isTrue => v.name } + val nodeVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTNode) => v.name } + val relVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTRelationship) => v.name } val nodeFrames = nodeVarNames.flatMap(result.nodeFrames).toSeq val relFrames = relVarNames.flatMap(result.relationshipFrames).toSeq diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index 161dd66860ad9..e811e943c2359 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -204,10 +204,6 @@ object SparkTable { df.dropDuplicates(colNames) } - override def withColumnsRenamed(columnRenamings: Map[String, String]): DataFrameTable = { - df.safeRenameColumns(columnRenamings) - } - override def cache(): DataFrameTable = { val planToCache = df.queryExecution.analyzed if (df.sparkSession.sharedState.cacheManager.lookupCachedData(planToCache).nonEmpty) { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala index 4d12e04ab709c..a9a155d81e756 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -10,7 +10,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame, functions} import org.opencypher.okapi.api.types._ import org.opencypher.okapi.api.value.CypherValue.{CypherList, CypherMap} -import org.opencypher.okapi.impl.exception.{IllegalArgumentException, IllegalStateException, NotImplementedException, UnsupportedOperationException} +import org.opencypher.okapi.impl.exception._ import org.opencypher.okapi.impl.temporal.TemporalTypesHelper._ import org.opencypher.okapi.impl.temporal.{Duration => DurationValue} import org.opencypher.okapi.ir.api.PropertyKey @@ -19,6 +19,8 @@ import org.opencypher.okapi.relational.impl.table.RecordHeader object ExprConversions { + final case class ExprConversionException(msg: String) extends InternalException(msg) + private val NULL_LIT: Column = functions.lit(null) private val TRUE_LIT: Column = functions.lit(true) @@ -63,15 +65,33 @@ object ExprConversions { expr match { - // Context based lookups - case p@Param(name) if p.cypherType.isInstanceOf[CTList] => - parameters(name) match { - case CypherList(l) => functions.array(l.unwrap.map(functions.lit): _*) - case notAList => throw IllegalArgumentException("a Cypher list", notAList) - } + case AliasExpr(innerExpr, _) => innerExpr.asSparkSQLExpr + + case Explode(list) => list.cypherType match { + case CTList(_) | CTListOrNull(_) => functions.explode(list.asSparkSQLExpr) + case CTNull => functions.explode(functions.lit(null).cast(ArrayType(NullType))) + case other => throw IllegalArgumentException("CTList", other) + } + + case e if e.cypherType == CTNull => NULL_LIT case Param(name) => - toSparkLiteral(parameters(name).unwrap) + expr.cypherType match { + case CTList(inner) => + if (inner == CTAny) { + throw ExprConversionException(s"List parameter with inner type $inner not supported") + } else { + functions.array(parameters(name).asInstanceOf[CypherList].value.unwrap.map(functions.lit): _*) + } + case _ => toSparkLiteral(parameters(name).unwrap) + } + + case ListLit(exprs) => + if (expr.cypherType == CTAny) { + throw ExprConversionException(s"List literal with inner type ${expr.cypherType} not supported") + } else { + functions.array(exprs.map(_.asSparkSQLExpr): _*) + } case Property(e, PropertyKey(key)) => e.cypherType.material match { @@ -79,10 +99,10 @@ object ExprConversions { if (inner.keySet.contains(key)) e.asSparkSQLExpr.getField(key) else functions.lit(null) case CTDate => - TemporalConversions.temporalAccessor[Date](e.asSparkSQLExpr, key) + temporalAccessor[Date](e.asSparkSQLExpr, key) case CTLocalDateTime => - TemporalConversions.temporalAccessor[Timestamp](e.asSparkSQLExpr, key) + temporalAccessor[Timestamp](e.asSparkSQLExpr, key) case CTDuration => TemporalUdfs.durationAccessor(key.toLowerCase).apply(e.asSparkSQLExpr) @@ -94,29 +114,17 @@ object ExprConversions { columnFor(expr) } - case IsNull(e) => - e.asSparkSQLExpr.isNull - - case IsNotNull(e) => - e.asSparkSQLExpr.isNotNull + case IsNull(e) => e.asSparkSQLExpr.isNull + case IsNotNull(e) => e.asSparkSQLExpr.isNotNull - case _: Var | _: Param | _: HasLabel | _: HasType | _: StartNode | _: EndNode => - columnFor(expr) - - case AliasExpr(innerExpr, _) => - innerExpr.asSparkSQLExpr - - // Literals - case ListLit(exprs) => - functions.array(exprs.map(_.asSparkSQLExpr): _*) + case _: Var | _: Param | _: HasLabel | _: HasType | _: StartNode | _: EndNode => columnFor(expr) - case NullLit(ct) => - NULL_LIT.cast(ct.toSparkType.get) + case NullLit(ct) => NULL_LIT.cast(ct.getSparkType) case LocalDateTime(dateExpr) => dateExpr match { case Some(e) => - val localDateTimeValue = TemporalConversions.resolveTemporalArgument(e) + val localDateTimeValue = resolveTemporalArgument(e) .map(parseLocalDateTime) .map(java.sql.Timestamp.valueOf) .map { @@ -132,7 +140,7 @@ object ExprConversions { case Date(dateExpr) => dateExpr match { case Some(e) => - val dateValue = TemporalConversions.resolveTemporalArgument(e) + val dateValue = resolveTemporalArgument(e) .map(parseDate) .map(java.sql.Date.valueOf) .orNull @@ -142,7 +150,7 @@ object ExprConversions { } case Duration(durationExpr) => - val durationValue = TemporalConversions.resolveTemporalArgument(durationExpr).map { + val durationValue = resolveTemporalArgument(durationExpr).map { case Left(m) => DurationValue(m.mapValues(_.toLong)).toCalendarInterval case Right(s) => DurationValue.parse(s).toCalendarInterval }.orNull @@ -162,22 +170,26 @@ object ExprConversions { col.isNotNull, functions.size(col).cast(LongType) ) - case CTNull => NULL_LIT case other => throw NotImplementedException(s"size() on values of type $other") } case Ands(exprs) => exprs.map(_.asSparkSQLExpr).foldLeft(TRUE_LIT)(_ && _) - case Ors(exprs) => exprs.map(_.asSparkSQLExpr).foldLeft(FALSE_LIT)(_ || _) + case In(lhs, rhs) if lhs.cypherType == CTNull => + val array = rhs.asSparkSQLExpr + functions + .when(functions.size(array) === 0, FALSE_LIT) + .otherwise(NULL_LIT) + case In(lhs, rhs) => - if (rhs.cypherType == CTNull || lhs.cypherType == CTNull) { - NULL_LIT.cast(BooleanType) - } else { - val element = lhs.asSparkSQLExpr - val array = rhs.asSparkSQLExpr - new Column(ArrayContains(element.expr, array.expr)) - } + val element = lhs.asSparkSQLExpr + val array = rhs.asSparkSQLExpr + functions + .when(functions.size(array) === 0, FALSE_LIT) + .when(array.isNull, NULL_LIT) + .when(element.isNull, NULL_LIT) + .otherwise(new Column(ArrayContains(array.expr, element.expr))) case LessThan(lhs, rhs) => compare(lt, lhs, rhs) case LessThanOrEqual(lhs, rhs) => compare(lteq, lhs, rhs) @@ -192,34 +204,45 @@ object ExprConversions { val regex: String = parameters(name).unwrap.toString prop.asSparkSQLExpr.rlike(regex) - // Arithmetics + // Arithmetic case Add(lhs, rhs) => val lhsCT = lhs.cypherType val rhsCT = rhs.cypherType - lhsCT.material -> rhsCT.material match { - case (_: CTList, _) => - throw UnsupportedOperationException("List concatenation is not supported") + if (rhsCT == CTNull || rhsCT == CTNull) { + NULL_LIT + } else { + lhsCT.material -> rhsCT.material match { + case (CTList(lhInner), CTList(rhInner)) => + if (lhInner.material == rhInner.material || lhInner == CTVoid || rhInner == CTVoid) { + functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + } else { + throw ExprConversionException(s"Lists of different inner types are not supported (${lhInner.material}, ${rhInner.material})") + } - case (_, _: CTList) => - throw UnsupportedOperationException("List concatenation is not supported") + case (CTList(inner), nonListType) if nonListType == inner.material || inner.material == CTVoid => + functions.concat(lhs.asSparkSQLExpr, functions.array(rhs.asSparkSQLExpr)) - case (CTString, _) if rhsCT.subTypeOf(CTNumber).maybeTrue => - functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr.cast(StringType)) + case (nonListType, CTList(inner)) if inner.material == nonListType || inner.material == CTVoid => + functions.concat(functions.array(lhs.asSparkSQLExpr), rhs.asSparkSQLExpr) - case (_, CTString) if lhsCT.subTypeOf(CTNumber).maybeTrue => - functions.concat(lhs.asSparkSQLExpr.cast(StringType), rhs.asSparkSQLExpr) + case (CTString, _) if rhsCT.subTypeOf(CTNumber) => + functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr.cast(StringType)) - case (CTString, CTString) => - functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + case (_, CTString) if lhsCT.subTypeOf(CTNumber) => + functions.concat(lhs.asSparkSQLExpr.cast(StringType), rhs.asSparkSQLExpr) - case (CTDate, CTDuration) => - TemporalUdfs.dateAdd(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + case (CTString, CTString) => + functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) - case _ => - lhs.asSparkSQLExpr + rhs.asSparkSQLExpr + case (CTDate, CTDuration) => + TemporalUdfs.dateAdd(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + + case _ => + lhs.asSparkSQLExpr + rhs.asSparkSQLExpr + } } - case Subtract(lhs, rhs) if lhs.cypherType.material.subTypeOf(CTDate).isTrue && rhs.cypherType.material.subTypeOf(CTDuration).isTrue => + case Subtract(lhs, rhs) if lhs.cypherType.material.subTypeOf(CTDate) && rhs.cypherType.material.subTypeOf(CTDuration) => TemporalUdfs.dateSubtract(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) case Subtract(lhs, rhs) => lhs.asSparkSQLExpr - rhs.asSparkSQLExpr @@ -227,8 +250,12 @@ object ExprConversions { case Multiply(lhs, rhs) => lhs.asSparkSQLExpr * rhs.asSparkSQLExpr case div@Divide(lhs, rhs) => (lhs.asSparkSQLExpr / rhs.asSparkSQLExpr).cast(div.cypherType.getSparkType) - // Functions + // Id functions + case Id(e) => e.asSparkSQLExpr + + // Functions + case _: MonotonicallyIncreasingId => functions.monotonically_increasing_id() case Exists(e) => e.asSparkSQLExpr.isNotNull case Labels(e) => e.cypherType match { @@ -242,7 +269,6 @@ object ExprConversions { .unzip val booleanLabelFlagColumn = functions.array(labelColumns: _*) get_node_labels(labelNames)(booleanLabelFlagColumn) - case CTNull => NULL_LIT case other => throw IllegalArgumentException("an expression with type CTNode, CTNodeOrNull, or CTNull", other) } @@ -298,19 +324,10 @@ object ExprConversions { header.endNodeFor(rel).asSparkSQLExpr case ToFloat(e) => e.asSparkSQLExpr.cast(DoubleType) - case ToInteger(e) => e.asSparkSQLExpr.cast(IntegerType) - case ToString(e) => e.asSparkSQLExpr.cast(StringType) - case ToBoolean(e) => e.asSparkSQLExpr.cast(BooleanType) - case Explode(list) => list.cypherType match { - case CTList(_) | CTListOrNull(_) => functions.explode(list.asSparkSQLExpr) - case CTNull => functions.explode(functions.lit(null).cast(ArrayType(NullType))) - case other => throw IllegalArgumentException("CTList", other) - } - case Trim(str) => functions.trim(str.asSparkSQLExpr) case LTrim(str) => functions.ltrim(str.asSparkSQLExpr) case RTrim(str) => functions.rtrim(str.asSparkSQLExpr) @@ -324,7 +341,6 @@ object ExprConversions { case Replace(original, search, replacement) => new Column(StringTranslate(original.asSparkSQLExpr.expr, search.asSparkSQLExpr.expr, replacement.asSparkSQLExpr.expr)) - case Substring(original, start, maybeLength) => val origCol = original.asSparkSQLExpr val startCol = start.asSparkSQLExpr + ONE_LIT @@ -359,24 +375,16 @@ object ExprConversions { case Sin(e) => functions.sin(e.asSparkSQLExpr) case Tan(e) => functions.tan(e.asSparkSQLExpr) - // Time functions case Timestamp() => functions.current_timestamp().cast(LongType) // Bit operations - case BitwiseAnd(lhs, rhs) => - lhs.asSparkSQLExpr.bitwiseAND(rhs.asSparkSQLExpr) - - case BitwiseOr(lhs, rhs) => - lhs.asSparkSQLExpr.bitwiseOR(rhs.asSparkSQLExpr) - - case ShiftLeft(value, IntegerLit(shiftBits)) => - functions.shiftLeft(value.asSparkSQLExpr, shiftBits.toInt) - - case ShiftRightUnsigned(value, IntegerLit(shiftBits)) => - functions.shiftRightUnsigned(value.asSparkSQLExpr, shiftBits.toInt) + case BitwiseAnd(lhs, rhs) => lhs.asSparkSQLExpr.bitwiseAND(rhs.asSparkSQLExpr) + case BitwiseOr(lhs, rhs) => lhs.asSparkSQLExpr.bitwiseOR(rhs.asSparkSQLExpr) + case ShiftLeft(value, IntegerLit(shiftBits)) => functions.shiftLeft(value.asSparkSQLExpr, shiftBits.toInt) + case ShiftRightUnsigned(value, IntegerLit(shiftBits)) => functions.shiftRightUnsigned(value.asSparkSQLExpr, shiftBits.toInt) // Pattern Predicate case ep: ExistsPatternExpr => ep.targetField.asSparkSQLExpr @@ -412,11 +420,9 @@ object ExprConversions { } case MapExpression(items) => expr.cypherType.material match { - case CTMap(inner) => + case CTMap(_) => val innerColumns = items.map { - case (key, innerExpr) => - val targetType = inner(key).toSparkType.get - innerExpr.asSparkSQLExpr.cast(targetType).as(key) + case (key, innerExpr) => innerExpr.asSparkSQLExpr.as(key) }.toSeq createStructColumn(innerColumns) case other => throw IllegalArgumentException("an expression of type CTMap", other) @@ -458,5 +464,6 @@ object ExprConversions { } } + } diff --git a/graph/cypher/src/test/resources/tck/failing_blacklist b/graph/cypher/src/test/resources/tck/failing_blacklist index 53f3c7a52477e..9db31748f2f36 100644 --- a/graph/cypher/src/test/resources/tck/failing_blacklist +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -16,7 +16,6 @@ Feature "VarLengthAcceptance": Scenario "Handling mixed relationship patterns an 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 concatenation of lists" 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" @@ -46,7 +45,6 @@ Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #2" Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #3" Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #4" Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #5" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in IN #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" @@ -150,7 +148,6 @@ 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 "Adding list properties in projection" 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" @@ -182,7 +179,6 @@ 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 "`properties()` on null" Feature "FunctionsAcceptance": Scenario "`reverse()`" Feature "FunctionsAcceptance": Scenario "`percentileDisc()` #1" Feature "FunctionsAcceptance": Scenario "`percentileDisc()` #2" @@ -208,7 +204,6 @@ Feature "Comparability": Scenario "Comparing across types yields null, except nu 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 "ColumnNameAcceptance": Scenario "Keeping used expression 4" Feature "AggregationAcceptance": Scenario "Handle aggregation on functions" Feature "AggregationAcceptance": Scenario "Aggregates in aggregates" Feature "AggregationAcceptance": Scenario "Multiple aggregates on same variable" @@ -216,18 +211,13 @@ 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 "AggregationAcceptance": Scenario "`max()` should aggregate strings" -Feature "AggregationAcceptance": Scenario "`min()` should aggregate strings" Feature "Aggregation": Scenario "`min()` over mixed values" Feature "Aggregation": Scenario "`min()` over list values" Feature "Aggregation": Scenario "`max()` over mixed values" -Feature "Aggregation": Scenario "`max()` over strings" Feature "Aggregation": Scenario "`max()` over mixed numeric values" -Feature "Aggregation": Scenario "`min()` over strings" Feature "Aggregation": Scenario "`max()` over list 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 null when LHS and RHS both ultimately contain null, even if LHS and RHS are of different types (nested list and flat list)" Feature "ListOperations": Scenario "IN should return false when matching a number with a string - list version" Feature "ListOperations": Scenario "IN should return false when types of LHS and RHS don't match - singleton list" Feature "ListOperations": Scenario "IN should return false when matching a number with a string" @@ -257,7 +247,6 @@ Feature "ListOperations": Scenario "IN should return true when LHS and RHS conta Feature "ListOperations": Scenario "IN should return true when types of LHS and RHS match - singleton list" Feature "ListOperations": Scenario "IN should return false when types of LHS and RHS don't match - list" Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - directed, one way" -Feature "ListOperations": Scenario "Concatenating lists of same type #2" 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" @@ -273,5 +262,22 @@ Feature "ListOperations": Scenario "IN should return true when both LHS and RHS 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 "Concatenating and returning the size of literal lists" -Feature "ListOperations": Scenario "Concatenating lists of same type #1" \ No newline at end of file +Feature "ListOperations": Scenario "List slice with negative range" +Feature "ListOperations": Scenario "List slice with singleton range" +Feature "ListOperations": Scenario "List slice" +Feature "ListOperations": Scenario "List slice with parameterised invalid range" +Feature "ListOperations": Scenario "List slice with implicit start" +Feature "ListOperations": Scenario "List slice with empty range" +Feature "ListOperations": Scenario "List slice with invalid range" +Feature "ListOperations": Scenario "List slice with parameterised range" +Feature "ListOperations": Scenario "List slice with null range #1" +Feature "ListOperations": Scenario "List slice with null range #2" +Feature "ListOperations": Scenario "List slice with null range #3" +Feature "ListOperations": Scenario "List slice with null range #4" +Feature "ListOperations": Scenario "List slice with null range #5" +Feature "ListOperations": Scenario "List slice with implicit end" +Feature "ListOperations": Scenario "List slice with exceeding range" +Feature "NullAcceptance": Scenario "Property existence check on null node" +Feature "NullOperator": Scenario "Property null check on null node" +Feature "NullOperator": Scenario "Property not null check on null node" +Feature "ColumnNameAcceptance": Scenario "Keeping used expression 4" \ No newline at end of file diff --git a/graph/cypher/src/test/resources/tck/temporal_blacklist b/graph/cypher/src/test/resources/tck/temporal_blacklist index 72dd5d62f51e7..cf491a6c39337 100644 --- a/graph/cypher/src/test/resources/tck/temporal_blacklist +++ b/graph/cypher/src/test/resources/tck/temporal_blacklist @@ -83,6 +83,7 @@ Feature "TemporalCreateAcceptance": Scenario "Should construct date time with de 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" @@ -360,9 +361,7 @@ 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 #1" Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #2" -Feature "TemporalArithmeticAcceptance": Scenario "Should add or subtract duration to or from date #3" Feature "TemporalCreateAcceptance": Scenario "Should store local time #1" Feature "TemporalCreateAcceptance": Scenario "Should store local time #2" Feature "TemporalCreateAcceptance": Scenario "Should store local time #3" @@ -918,5 +917,24 @@ Feature "TemporalCreateAcceptance": Scenario "Should construct date time with of 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" \ No newline at end of file +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 From bfbd0235de4236d6562ecc474478e66711128f0b Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 14 Mar 2019 12:02:49 +0100 Subject: [PATCH 043/123] Expose schema information directly on PropertyGraph Offer accessors to Node-/RelationshipFrames Co-authored-by: Philip Stutz --- .../spark/graph/api/PropertyGraph.scala | 31 +++++++++++++++++++ .../graph/cypher/SparkCypherSession.scala | 8 ++--- .../adapters/RelationalGraphAdapter.scala | 29 ++++++++++++++--- 3 files changed, 58 insertions(+), 10 deletions(-) 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 index ebd98330172f1..06c1c9001eaf1 100644 --- 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 @@ -12,6 +12,21 @@ import org.apache.spark.sql.DataFrame */ trait PropertyGraph { + /** + * Returns all labels occurring on any node in the graph. + */ + def labels: Set[String] = labelSets.flatten + + /** + * Returns all distinct label sets occurring on nodes in the graph. + */ + def labelSets: Set[Set[String]] + + /** + * Returns all relationship types occurring on relationships in the graph. + */ + def relationshipTypes: Set[String] + /** * The session in which this graph is managed. */ @@ -25,6 +40,22 @@ trait PropertyGraph { */ def cypher(query: String, parameters: Map[String, Any] = Map.empty): CypherResult = cypherSession.cypher(this, query, parameters) + /** + * Returns the [[NodeFrame]] for a given node label set. + * + * @param labelSet Label set used for [[NodeFrame]] lookup + * @return [[NodeFrame]] for the given label set + */ + def nodeFrame(labelSet: Set[String]): NodeFrame + + /** + * Returns the [[RelationshipFrame]] for a given relationship type. + * + * @param relationshipType Relationship type used for [[RelationshipFrame]] lookup + * @return [[RelationshipFrame]] for the given relationship type + */ + def relationshipFrame(relationshipType: String): RelationshipFrame + /** * Returns a [[DataFrame]] that contains a row for each node in this graph. */ diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index f51e87aaf0b4b..0e49a5e980ef9 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -3,7 +3,6 @@ package org.apache.spark.graph.cypher import org.apache.spark.graph.api._ import org.apache.spark.graph.api.io.{ReaderConfig, WriterConfig} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter import org.apache.spark.graph.cypher.io.ReadWriteGraph._ import org.apache.spark.sql.SparkSession @@ -48,10 +47,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph = { require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") - val nodeTables = nodes.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } - val relTables = relationships.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } - - RelationalGraphAdapter(this, graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*)) + RelationalGraphAdapter(this, nodes, relationships) } override def createGraph(result: CypherResult): PropertyGraph = { @@ -107,7 +103,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) private def toRelationalGraph(graph: PropertyGraph): RelationalCypherGraph[DataFrameTable] = { graph match { - case RelationalGraphAdapter(_, relGraph) => relGraph + case adapter: RelationalGraphAdapter => adapter.graph case other => throw IllegalArgumentException( expected = "A graph that has been created by `SparkCypherSession.createGraph`", actual = other.getClass.getSimpleName diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala index 98100e27ded01..4d647a6aba7f3 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala @@ -1,15 +1,36 @@ package org.apache.spark.graph.cypher.adapters -import org.apache.spark.graph.api.{CypherSession, PropertyGraph} +import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.graph.cypher.adapters.MappingAdapter._ +import org.apache.spark.graph.cypher.{SparkCypherSession, SparkEntityTable} import org.apache.spark.sql.DataFrame -import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph case class RelationalGraphAdapter( - cypherSession: CypherSession, - graph: RelationalCypherGraph[DataFrameTable]) extends PropertyGraph { + cypherSession: SparkCypherSession, + nodeFrames: Seq[NodeFrame], + relationshipFrames: Seq[RelationshipFrame]) extends PropertyGraph { + + private [graph] lazy val graph = { + val nodeTables = nodeFrames.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } + val relTables = relationshipFrames.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } + cypherSession.graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*) + } + + private lazy val _nodeFrame: Map[Set[String], NodeFrame] = nodeFrames.map(nf => nf.labels -> nf).toMap + + private lazy val _relationshipFrame: Map[String, RelationshipFrame] = relationshipFrames.map(rf => rf.relationshipType -> rf).toMap override def nodes: DataFrame = graph.nodes("n").table.df override def relationships: DataFrame = graph.relationships("r").table.df + + override def labelSets: Set[Set[String]] = graph.schema.labelCombinations.combos + + override def relationshipTypes: Set[String] = graph.schema.relationshipTypes + + override def nodeFrame(labelSet: Set[String]): NodeFrame = _nodeFrame(labelSet) + + override def relationshipFrame(relationshipType: String): RelationshipFrame = _relationshipFrame(relationshipType) + } From e283ab914dd957cf33c548f8086003d255aa0b9d Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Thu, 14 Mar 2019 14:53:47 +0000 Subject: [PATCH 044/123] Fix Scala warning --- .../org/apache/spark/graph/cypher/SparkCypherRecords.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala index 7560444c014ee..d8a4212f6d986 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala @@ -15,7 +15,7 @@ import org.opencypher.okapi.relational.impl.table._ import scala.collection.JavaConverters._ -case class SparkCypherRecordsFactory(implicit caps: SparkCypherSession) extends RelationalCypherRecordsFactory[DataFrameTable] { +case class SparkCypherRecordsFactory()(implicit caps: SparkCypherSession) extends RelationalCypherRecordsFactory[DataFrameTable] { override type Records = SparkCypherRecords From b3f5685f6eb27a6665227616dc47e3654b100f08 Mon Sep 17 00:00:00 2001 From: Philip Stutz Date: Thu, 14 Mar 2019 14:55:37 +0000 Subject: [PATCH 045/123] Minor cleanups --- .../spark/graph/cypher/SparkCypherRecords.scala | 13 ------------- .../graph/cypher/conversions/RowConversion.scala | 1 - .../apache/spark/graph/cypher/udfs/LegacyUdfs.scala | 1 - 3 files changed, 15 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala index d8a4212f6d986..5cf1eac819ea4 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala @@ -46,18 +46,6 @@ case class SparkCypherRecordsFactory()(implicit caps: SparkCypherSession) extend SparkCypherRecords(header, table, displayNames) } - // /** - // * Wraps a Spark SQL table (DataFrame) in a CAPSRecords, making it understandable by Cypher. - // * - // * @param df table to wrap. - // * @param caps session to which the resulting CAPSRecords is tied. - // * @return a Cypher table. - // */ - // private[spark] def wrap(df: DataFrame)(implicit caps: SparkCypherSession): CAPSRecords = { - // val compatibleDf = df.withCypherCompatibleTypes - // CAPSRecords(compatibleDf.schema.toRecordHeader, compatibleDf) - // } - private case class EmptyRow() } @@ -112,4 +100,3 @@ trait RecordBehaviour extends RelationalCypherRecords[DataFrameTable] { table.df.map(RowConversion(header.exprToColumn.toSeq)) } } - diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala index 29825a3b367f3..c1649021d7e44 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala @@ -130,4 +130,3 @@ final case class RowConversion(exprToColumn: Seq[(Expr, String)]) extends (Row = CypherList(values) } } - diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala index d08d0f9a064a1..9bd5d6315d791 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala @@ -27,4 +27,3 @@ object LegacyUdfs { case (key, value) if value != null => key } } - From c21fbb1eb98c4680e4790607ac7c7c866ca1530d Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 22 Mar 2019 16:47:03 +0100 Subject: [PATCH 046/123] Use column name conventions for node/relationship DataFrame Adds support for creating a PropertyGraph from two DataFrames following the column naming conventions. --- .../spark/graph/api/CypherSession.scala | 23 +++++++++++- .../graph/cypher/SparkCypherSession.scala | 33 ++++++++++++++++- .../adapters/RelationalGraphAdapter.scala | 36 +++++++++++++++++-- .../spark/graph/cypher/BasicMatchSuite.scala | 30 +++++++++++++++- 4 files changed, 117 insertions(+), 5 deletions(-) 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 index 6a90d874254bb..a3fcf7f0f07c8 100644 --- 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 @@ -1,7 +1,7 @@ package org.apache.spark.graph.api import org.apache.spark.graph.api.io.{PropertyGraphReader, PropertyGraphWriter, ReaderConfig, WriterConfig} -import org.apache.spark.sql.{SaveMode, SparkSession} +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} /** * Allows for creating [[PropertyGraph]] instances and running Cypher-queries on them. @@ -43,6 +43,27 @@ trait CypherSession { */ def createGraph(result: CypherResult): PropertyGraph + /** + * Creates a [[PropertyGraph]] from nodes and relationships. + * + * The given DataFrames need to adhere to 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) + * }}} + * + * @param nodes node [[DataFrame]] + * @param relationships relationship [[DataFrame]] + */ + def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph + /** * Returns a [[PropertyGraphWriter]] for `graph`. */ diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 0e49a5e980ef9..debb9721df27c 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -5,7 +5,7 @@ import org.apache.spark.graph.api.io.{ReaderConfig, WriterConfig} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter import org.apache.spark.graph.cypher.io.ReadWriteGraph._ -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{DataFrame, SparkSession, functions} import org.opencypher.okapi.api.types.{CTNode, CTRelationship} import org.opencypher.okapi.api.value.CypherValue.CypherMap import org.opencypher.okapi.impl.exception.{IllegalArgumentException, UnsupportedOperationException} @@ -69,6 +69,37 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) createGraph(nodeFrames, relFrames) } + override def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { + val idColumn = "$ID" + val sourceIdColumn = "$SOURCE_ID" + val targetIdColumn = "$TARGET_ID" + + val labelColumns = nodes.columns.filter(_.startsWith(":")).toSet + val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns).map(col => col -> col).toMap + + val trueLit = functions.lit(true) + val falseLit = functions.lit(false) + + val nodeFrames = labelColumns.subsets().map { labelSet => + val predicate = labelColumns.map { + case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) === trueLit + case labelColumn => nodes.col(labelColumn) === falseLit + }.reduce(_ && _) + + NodeFrame(nodes.filter(predicate), idColumn, labelSet.map(_.substring(1)), nodeProperties) + } + + val relTypeColumns = relationships.columns.filter(_.startsWith(":")).toSet + val relProperties = (relationships.columns.toSet - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns).map(col => col -> col).toMap + val relFrames = relTypeColumns.map { relTypeColumn => + val predicate = relationships.col(relTypeColumn) === trueLit + + RelationshipFrame(relationships.filter(predicate), idColumn, sourceIdColumn, targetIdColumn, relTypeColumn.substring(1), relProperties) + } + + createGraph(nodeFrames.toSeq, relFrames.toSeq) + } + def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala index 4d647a6aba7f3..05e633c477982 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala @@ -5,6 +5,8 @@ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.{SparkCypherSession, SparkEntityTable} import org.apache.spark.sql.DataFrame +import org.opencypher.okapi.api.types.{CTNode, CTRelationship} +import org.opencypher.okapi.ir.api.expr.Var case class RelationalGraphAdapter( cypherSession: SparkCypherSession, @@ -21,9 +23,39 @@ case class RelationalGraphAdapter( private lazy val _relationshipFrame: Map[String, RelationshipFrame] = relationshipFrames.map(rf => rf.relationshipType -> rf).toMap - override def nodes: DataFrame = graph.nodes("n").table.df + override def nodes: DataFrame = { + val nodeVar = Var("n")(CTNode) + val nodes = graph.nodes(nodeVar.name) - override def relationships: DataFrame = graph.relationships("r").table.df + 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: DataFrame = { + 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 labelSets: Set[Set[String]] = graph.schema.labelCombinations.combos diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 95f524aa1f152..e25b2ab58c35c 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -59,5 +59,33 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { val berkeleyGraph2 = cypherSession.createGraph(result) berkeleyGraph2.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() } - + + test("round trip example using column name conventions") { + + val nodes: DataFrame = spark.createDataFrame(Seq( + (0, true, false, Some("Alice"), Some(42), None), + (1, true, false, Some("Bob"), Some(23), None), + (2, true, false, Some("Eve"), Some(19), None), + (3, false, true, None, None, Some("UC Berkeley")), + (4, false, true, None, None, Some("Stanford")) + )).toDF("$ID", ":Student", ":University", "name", "age", "title") + + val relationships: DataFrame = spark.createDataFrame(Seq( + (0, 0, 1, true, false), + (1, 0, 2, true, false), + (2, 0, 2, false, true), + (3, 1, 3, false, true), + (4, 2, 2, false, true), + )).toDF("$ID", "$SOURCE_ID", "$TARGET_ID", ":KNOWS", ":STUDY_AT") + + val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) + graph1.nodes.show() + graph1.relationships.show() + + graph1.cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() + + val graph2: PropertyGraph = cypherSession.createGraph(graph1.nodes, graph1.relationships) + graph2.nodes.show() + graph2.relationships.show() + } } From 415a57714637ac4a7d8450d848d128e7cbc13f9b Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 22 Mar 2019 16:55:33 +0100 Subject: [PATCH 047/123] Update documentation for PropertyGraph.[nodes/relationships] --- .../apache/spark/graph/api/PropertyGraph.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) 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 index 06c1c9001eaf1..85be31aee040b 100644 --- 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 @@ -58,11 +58,29 @@ trait PropertyGraph { /** * Returns a [[DataFrame]] that contains a row for each node in this graph. + * + * The DataFrame adheres to column naming conventions: + * + * {{{ + * Id column: `$ID` + * Label columns: `:{LABEL_NAME}` + * Property columns: `{Property_Key}` + * }}} */ def nodes: DataFrame /** * Returns a [[DataFrame]] that contains a row for each relationship in this graph. + * + * The DataFrame adheres to column naming conventions: + * + * {{{ + * Id column: `$ID` + * SourceId column: `$SOURCE_ID` + * TargetId column: `$TARGET_ID` + * RelType columns: `:{REL_TYPE}` + * Property columns: `{Property_Key}` + * }}} */ def relationships: DataFrame From 33fe7fd757302ccabd68806a186fb864fc315ea5 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 25 Mar 2019 12:01:59 +0100 Subject: [PATCH 048/123] Introduce CypherResult.graph to allow for round trip examples Co-authored-by: Philip Stutz --- .../apache/spark/graph/api/CypherResult.scala | 13 +++++++++++ .../spark/graph/api/CypherSession.scala | 7 ------ .../graph/cypher/SparkCypherResult.scala | 15 ++++++++++-- .../graph/cypher/SparkCypherSession.scala | 23 +------------------ .../adapters/RelationalGraphAdapter.scala | 10 +++++--- .../spark/graph/cypher/BasicMatchSuite.scala | 22 ++++++++++++++---- 6 files changed, 51 insertions(+), 39 deletions(-) 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 index 4fb9a4544a3e3..dcd06ba04a8b8 100644 --- 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 @@ -19,6 +19,19 @@ trait CypherResult { */ def df: DataFrame + // TODO: determine what happens if there are dangling relationships (create empty nodes or filter such relationships) + /** + * Creates a [[PropertyGraph]] from nodes and relationship present in this result. + * + * @example + * {{{ + * val result = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") + * // returns the original graph + * val graph2 = result.graph + * }}} + */ + def graph: PropertyGraph + /** * Extracts nodes that are specified as a return item. * 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 index a3fcf7f0f07c8..04a3a8ca8e6b6 100644 --- 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 @@ -36,13 +36,6 @@ trait CypherSession { */ def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph - /** - * Creates a [[PropertyGraph]] from nodes and relationship present in the given [[CypherResult]]. - * - * @param result a [[CypherResult]] containing nodes and/or relationships - */ - def createGraph(result: CypherResult): PropertyGraph - /** * Creates a [[PropertyGraph]] from nodes and relationships. * diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index b9c6521345dd9..f56a2364965de 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -1,6 +1,6 @@ package org.apache.spark.graph.cypher -import org.apache.spark.graph.api.{CypherResult, NodeFrame, RelationshipFrame} +import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.sql.{DataFrame, functions} import org.opencypher.okapi.api.schema.Schema @@ -10,12 +10,23 @@ import org.opencypher.okapi.ir.api.expr._ import org.opencypher.okapi.relational.api.table.RelationalCypherRecords import org.opencypher.okapi.relational.impl.table.RecordHeader -case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable], schema: Schema) extends CypherResult { +case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable], schema: Schema, session: SparkCypherSession) extends CypherResult { override val df: DataFrame = relationalTable.table.df private val header: RecordHeader = relationalTable.header + override def graph: PropertyGraph = { + val entityVars = relationalTable.header.entityVars + val nodeVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTNode) => v.name } + val relVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTRelationship) => v.name } + + val nodeFrames = nodeVarNames.flatMap(this.nodeFrames).toSeq + val relFrames = relVarNames.flatMap(this.relationshipFrames).toSeq + + session.createGraph(nodeFrames, relFrames) + } + // TODO: Error handling override def nodeFrames(varName: String): Seq[NodeFrame] = { val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index debb9721df27c..02e89909de530 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -6,7 +6,6 @@ import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter import org.apache.spark.graph.cypher.io.ReadWriteGraph._ import org.apache.spark.sql.{DataFrame, SparkSession, functions} -import org.opencypher.okapi.api.types.{CTNode, CTRelationship} 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} @@ -46,29 +45,9 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) } override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph = { - require(nodes.nonEmpty, "Creating a graph requires at least one NodeDataFrame") RelationalGraphAdapter(this, nodes, relationships) } - override def createGraph(result: CypherResult): PropertyGraph = { - val sparkCypherResult = result match { - case r: SparkCypherResult => r - case other => throw IllegalArgumentException( - expected = "A result that has been created by `SparkCypherSession.cypher`", - actual = other.getClass.getSimpleName - ) - } - - val entityVars = sparkCypherResult.relationalTable.header.entityVars - val nodeVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTNode) => v.name } - val relVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTRelationship) => v.name } - - val nodeFrames = nodeVarNames.flatMap(result.nodeFrames).toSeq - val relFrames = relVarNames.flatMap(result.relationshipFrames).toSeq - - createGraph(nodeFrames, relFrames) - } - override def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { val idColumn = "$ID" val sourceIdColumn = "$SOURCE_ID" @@ -104,7 +83,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { val relationalGraph = toRelationalGraph(graph) - SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema) + SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema, this) } override private[spark] def readGraph(config: ReaderConfig): PropertyGraph = { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala index 05e633c477982..6c96395912f11 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala @@ -14,9 +14,13 @@ case class RelationalGraphAdapter( relationshipFrames: Seq[RelationshipFrame]) extends PropertyGraph { private [graph] lazy val graph = { - val nodeTables = nodeFrames.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } - val relTables = relationshipFrames.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } - cypherSession.graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*) + if (nodeFrames.isEmpty) { + cypherSession.graphs.empty + } else { + val nodeTables = nodeFrames.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } + val relTables = relationshipFrames.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } + cypherSession.graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*) + } } private lazy val _nodeFrame: Map[Set[String], NodeFrame] = nodeFrames.map(nf => nf.labels -> nf).toMap diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index e25b2ab58c35c..054b1aeed3fa2 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -56,7 +56,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() // Option 2: Use CypherResult to create a new PropertyGraph - val berkeleyGraph2 = cypherSession.createGraph(result) + val berkeleyGraph2 = result.graph berkeleyGraph2.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() } @@ -82,10 +82,22 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { graph1.nodes.show() graph1.relationships.show() - graph1.cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() + val result = graph1.cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u") - val graph2: PropertyGraph = cypherSession.createGraph(graph1.nodes, graph1.relationships) - graph2.nodes.show() - graph2.relationships.show() + def pageRank(g: PropertyGraph): PropertyGraph = { + // some pagerank implementation + g + } + + val graph2 = result.graph + val graph3 = pageRank(graph2) + + graph3.nodes.show() + graph3.relationships.show() + result.df.show() + + val graph4: PropertyGraph = cypherSession.createGraph(graph3.nodes, graph3.relationships) + graph4.nodes.show() + graph4.relationships.show() } } From 80a456d7f8fb51e8f7dadce82d5795f3e4859032 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 25 Mar 2019 15:11:47 +0100 Subject: [PATCH 049/123] Add PoC for graph algorithm support using GraphX PageRank Co-authored-by: Philip Stutz --- .../graph/cypher/SparkCypherResult.scala | 4 +- .../graph/cypher/algos/GraphAlgorithms.scala | 50 ++++++++++++++++ .../spark/graph/cypher/BasicMatchSuite.scala | 59 +++++++++---------- 3 files changed, 79 insertions(+), 34 deletions(-) create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index f56a2364965de..3a2356763033f 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -33,14 +33,14 @@ case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameT val idColumn = header.column(nodeVar) val possibleLabels = header.labelsFor(nodeVar).map(_.label.name) - val labelCombinations = schema.combinationsFor(possibleLabels).toSeq + val labelCombinations = possibleLabels.flatMap(label => schema.combinationsFor(Set(label))) val labelToColumns = header.labelsFor(nodeVar).map(expr => expr.label.name -> header.column(expr)).toMap val propertyToColumns = header.propertiesFor(nodeVar).map(expr => expr.key.name -> header.column(expr)).toMap val allLabelColumns = labelToColumns.values.toSet - labelCombinations.map { labels => + labelCombinations.toSeq.map { labels => val trueLabels = labels.map(labelToColumns) val falseLabels = allLabelColumns -- trueLabels diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala new file mode 100644 index 0000000000000..24daac4431c40 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala @@ -0,0 +1,50 @@ +package org.apache.spark.graph.cypher.algos + +import org.apache.spark.graph.api.PropertyGraph +import org.apache.spark.graphx.{Edge, Graph} +import org.apache.spark.sql.types.BinaryType + +object GraphAlgorithms { + + implicit class PageRankGraph(graph: PropertyGraph) { + + def withPageRank(tolerance: Double = 0.01, resetProbability: Double = 0.15): PropertyGraph = { + // Create GraphX compatible RDDs from nodes and relationships + val graphXNodeRDD = graph + .nodes + .select("$ID") + .rdd.map(row => row.getAs[Array[Byte]](0).toLong -> null) + + val graphXRelRDD = graph + .relationships + .select("$SOURCE_ID", "$TARGET_ID") + .rdd.map(row => Edge(row.getAs[Array[Byte]](0).toLong, row.getAs[Array[Byte]](1).toLong, ())) + + // Compute Page Rank via GraphX + val graphX = Graph(graphXNodeRDD, graphXRelRDD) + val ranks = graphX.pageRank(tolerance, resetProbability).vertices + + // Convert RDD to DataFrame + val rankDf = graph.cypherSession.sparkSession.createDataFrame(ranks) + val rankTable = rankDf.select( + rankDf.col("_1").cast(BinaryType).as("$ID"), + rankDf.col("_2").as("pageRank")) + + graph.cypherSession.createGraph(graph.nodes.join(rankTable, "$ID"), graph.relationships) + } + } + + implicit class ByteArrayConversion(val a: Array[Byte]) extends AnyVal { + def toLong: Long = { + var result = a(0).toLong << 56 + result |= a(1).toLong << 48 + result |= a(2).toLong << 40 + result |= a(3).toLong << 32 + result |= a(4).toLong << 24 + result |= a(5).toLong << 16 + result |= a(6).toLong << 8 + result |= a(7).toLong + result + } + } +} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 054b1aeed3fa2..61981398f1598 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -2,6 +2,7 @@ package org.apache.spark.graph.cypher import org.apache.spark.SparkFunSuite import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.graph.cypher.algos.GraphAlgorithms._ import org.apache.spark.sql.DataFrame class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { @@ -30,7 +31,7 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("create property graph from query results") { val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, "Alice", 42), Tuple3(1, "Bob", 23), Tuple3(2, "Eve", 19))).toDF("id", "name", "age") - val universityData: DataFrame = spark.createDataFrame(Seq(2 -> "UC Berkeley", 3-> "Stanford")).toDF("id", "title") + val universityData: DataFrame = spark.createDataFrame(Seq(2 -> "UC Berkeley", 3 -> "Stanford")).toDF("id", "title") val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, 0, 1), Tuple3(1, 0, 2))).toDF("id", "source", "target") val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(2, 0, 2), Tuple3(3, 1, 3), Tuple3(4, 2, 2))).toDF("id", "source", "target") val personDataFrame: NodeFrame = NodeFrame(df = personData, idColumn = "id", labels = Set("Student")) @@ -63,41 +64,35 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("round trip example using column name conventions") { val nodes: DataFrame = spark.createDataFrame(Seq( - (0, true, false, Some("Alice"), Some(42), None), - (1, true, false, Some("Bob"), Some(23), None), - (2, true, false, Some("Eve"), Some(19), None), - (3, false, true, None, None, Some("UC Berkeley")), - (4, false, true, None, None, Some("Stanford")) + (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") val relationships: DataFrame = spark.createDataFrame(Seq( - (0, 0, 1, true, false), - (1, 0, 2, true, false), - (2, 0, 2, false, true), - (3, 1, 3, false, true), - (4, 2, 2, false, true), + (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") - val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) - graph1.nodes.show() - graph1.relationships.show() - - val result = graph1.cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u") - - def pageRank(g: PropertyGraph): PropertyGraph = { - // some pagerank implementation - g - } - - val graph2 = result.graph - val graph3 = pageRank(graph2) - - graph3.nodes.show() - graph3.relationships.show() - result.df.show() - - val graph4: PropertyGraph = cypherSession.createGraph(graph3.nodes, graph3.relationships) - graph4.nodes.show() - graph4.relationships.show() + cypherSession.createGraph(nodes, relationships) + .cypher( + """|MATCH (student:Student)-[:STUDY_AT]->(uni:University), + | (student)-[knows:KNOWS]-(:Student) + |WHERE uni.title = 'UC Berkeley' + |RETURN student, knows""".stripMargin) + .graph + .withPageRank(tolerance = 0.02) + .cypher("MATCH (n:Student) RETURN n.name, n.pageRank ORDER BY n.pageRank DESC") + .df.show() } } From 119cf088d49183760688f60c0a2f32254b2db94c Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 25 Mar 2019 15:34:01 +0100 Subject: [PATCH 050/123] Bring back column naming convention example Co-authored-by: Philip Stutz --- .../spark/graph/cypher/BasicMatchSuite.scala | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 61981398f1598..81255c3de06dc 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -63,6 +63,35 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { test("round trip example using column name conventions") { + val nodes: DataFrame = spark.createDataFrame(Seq( + (0, true, false, Some("Alice"), Some(42), None), + (1, true, false, Some("Bob"), Some(23), None), + (2, true, false, Some("Eve"), Some(19), None), + (3, false, true, None, None, Some("UC Berkeley")), + (4, false, true, None, None, Some("Stanford")) + )).toDF("$ID", ":Student", ":University", "name", "age", "title") + + val relationships: DataFrame = spark.createDataFrame(Seq( + (0, 0, 1, true, false), + (1, 0, 2, true, false), + (2, 0, 2, false, true), + (3, 1, 3, false, true), + (4, 2, 2, false, true), + )).toDF("$ID", "$SOURCE_ID", "$TARGET_ID", ":KNOWS", ":STUDY_AT") + + val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) + graph1.nodes.show() + graph1.relationships.show() + + graph1.cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() + + val graph2: PropertyGraph = cypherSession.createGraph(graph1.nodes, graph1.relationships) + graph2.nodes.show() + graph2.relationships.show() + } + + test("query composition + graph analytics workflow") { + val nodes: DataFrame = spark.createDataFrame(Seq( (0L, true, false, Some("Alice"), Some(42), None), (1L, true, false, Some("Bob"), Some(23), None), From 77ab7b4a06730627d7a5faac144c1d265917e9a4 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 25 Mar 2019 15:53:56 +0100 Subject: [PATCH 051/123] Add example for retaining user ids Co-authored-by: Philip Stutz --- .../spark/graph/cypher/BasicMatchSuite.scala | 72 ++++++++----------- 1 file changed, 29 insertions(+), 43 deletions(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 81255c3de06dc..63b56e4904d79 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -62,57 +62,22 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { } test("round trip example using column name conventions") { - - val nodes: DataFrame = spark.createDataFrame(Seq( - (0, true, false, Some("Alice"), Some(42), None), - (1, true, false, Some("Bob"), Some(23), None), - (2, true, false, Some("Eve"), Some(19), None), - (3, false, true, None, None, Some("UC Berkeley")), - (4, false, true, None, None, Some("Stanford")) - )).toDF("$ID", ":Student", ":University", "name", "age", "title") - - val relationships: DataFrame = spark.createDataFrame(Seq( - (0, 0, 1, true, false), - (1, 0, 2, true, false), - (2, 0, 2, false, true), - (3, 1, 3, false, true), - (4, 2, 2, false, true), - )).toDF("$ID", "$SOURCE_ID", "$TARGET_ID", ":KNOWS", ":STUDY_AT") - val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) - graph1.nodes.show() - graph1.relationships.show() - - graph1.cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() - val graph2: PropertyGraph = cypherSession.createGraph(graph1.nodes, graph1.relationships) graph2.nodes.show() graph2.relationships.show() } - test("query composition + graph analytics workflow") { + test("example for retaining user ids") { + val nodesWithRetainedId = nodes.withColumn("retainedId", nodes.col("$ID")) + val relsWithRetainedId = relationships.withColumn("retainedId", relationships.col("$ID")) - val nodes: DataFrame = 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") - - val relationships: DataFrame = 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") + cypherSession + .createGraph(nodesWithRetainedId, relsWithRetainedId) + .cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() + } + test("query composition + graph analytics workflow") { cypherSession.createGraph(nodes, relationships) .cypher( """|MATCH (student:Student)-[:STUDY_AT]->(uni:University), @@ -124,4 +89,25 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { .cypher("MATCH (n:Student) RETURN n.name, n.pageRank ORDER BY n.pageRank DESC") .df.show() } + + lazy val nodes: DataFrame = 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: DataFrame = 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") } From 0731b0cd7064d61a35e9fdea94e055031d5b0234 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 25 Mar 2019 16:12:43 +0100 Subject: [PATCH 052/123] Simplify save/load with default parquet format Co-authored-by: Philip Stutz --- .../spark/graph/api/CypherSession.scala | 20 +----- .../spark/graph/api/PropertyGraph.scala | 8 +-- .../graph/api/io/PropertyGraphReader.scala | 45 -------------- .../graph/api/io/PropertyGraphWriter.scala | 61 ------------------- .../graph/cypher/SparkCypherSession.scala | 21 ++++--- .../graph/cypher/io/ReadWriteGraph.scala | 11 ++-- .../graph/cypher/PropertyGraphReadWrite.scala | 16 +---- 7 files changed, 26 insertions(+), 156 deletions(-) delete mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala delete mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala 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 index 04a3a8ca8e6b6..25ac8e19c7f2d 100644 --- 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 @@ -1,6 +1,5 @@ package org.apache.spark.graph.api -import org.apache.spark.graph.api.io.{PropertyGraphReader, PropertyGraphWriter, ReaderConfig, WriterConfig} import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} /** @@ -58,23 +57,10 @@ trait CypherSession { def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph /** - * Returns a [[PropertyGraphWriter]] for `graph`. + * Loads a [[PropertyGraph]] from the given location. */ - def write(graph: PropertyGraph): PropertyGraphWriter = PropertyGraphWriter(graph, WriterConfig( - path = ".", - SaveMode.ErrorIfExists, - sparkSession.sessionState.conf.defaultDataSourceName) - ) + def load(path: String): PropertyGraph - /** - * Returns a [[PropertyGraphReader]]. - */ - def read: PropertyGraphReader = PropertyGraphReader(this, ReaderConfig( - path = ".", - sparkSession.sessionState.conf.defaultDataSourceName)) - - private[spark] def readGraph(config: ReaderConfig): PropertyGraph - - private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit + private[spark] def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit } 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 index 85be31aee040b..98abae0932b69 100644 --- 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 @@ -1,7 +1,6 @@ package org.apache.spark.graph.api -import org.apache.spark.graph.api.io.PropertyGraphWriter -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SaveMode} /** * A Property Graph as defined by the openCypher Property Graph Model. @@ -85,8 +84,9 @@ trait PropertyGraph { def relationships: DataFrame /** - * Returns a [[PropertyGraphWriter]]. + * Saves this graph to the given location. */ - def write: PropertyGraphWriter = cypherSession.write(this) + def save(path: String, saveMode: SaveMode = SaveMode.ErrorIfExists): Unit = + cypherSession.save(this, path, saveMode) } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala deleted file mode 100644 index cffa149bc33a0..0000000000000 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphReader.scala +++ /dev/null @@ -1,45 +0,0 @@ -package org.apache.spark.graph.api.io - -import org.apache.spark.graph.api.{CypherSession, PropertyGraph} - -private[spark] case class ReaderConfig( - path: String, - source: String -) - -case class PropertyGraphReader( - session: CypherSession, - config: ReaderConfig -) { - - /** - * Saves the content of the `PropertyGraph` at the specified path. - */ - def read(path: String): PropertyGraph = { - session.readGraph(config.copy(path = path)) - } - - /** - * Saves the content of the `DataFrame` in Parquet format at the specified path. - */ - def parquet(path: String): PropertyGraph = { - format("parquet").read(path) - } - - /** - * Saves the content of the `DataFrame` in ORC format at the specified path. - * - * @note Currently, this method can only be used after enabling Hive support - */ - def orc(path: String): PropertyGraph = { - format("orc").read(path) - } - - /** - * Specifies the underlying output data source. Built-in options include "parquet", "json", etc. - */ - private[spark] def format(source: String): PropertyGraphReader = { - copy(config = config.copy(source = source)) - } - -} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala deleted file mode 100644 index 03eec798dcb4f..0000000000000 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/io/PropertyGraphWriter.scala +++ /dev/null @@ -1,61 +0,0 @@ -package org.apache.spark.graph.api.io - -import org.apache.spark.graph.api.PropertyGraph -import org.apache.spark.sql.SaveMode - -private[spark] case class WriterConfig( - path: String, - mode: SaveMode, - source: String -) - -case class PropertyGraphWriter( - graph: PropertyGraph, - config: WriterConfig -) { - - /** - * Saves the content of the `PropertyGraph` at the specified path. - */ - def save(path: String): Unit = { - copy(config = config.copy(path = path)).save() - } - - /** - * Specifies the behavior when data or table already exists. Options include: - *
    - *
  • `SaveMode.Overwrite`: overwrite the existing data.
  • - *
  • `SaveMode.Append`: append the data.
  • - *
  • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
  • - *
  • `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
  • - *
- */ - def mode(saveMode: SaveMode): PropertyGraphWriter = { - copy(config = config.copy(mode = saveMode)) - } - - private[spark] def format(source: String): PropertyGraphWriter = { - copy(config = config.copy(source = source)) - } - - /** - * Saves the content of the `PropertyGraph` in Parquet format at the specified path. - */ - def parquet(path: String): Unit = { - format("parquet").save(path) - } - - /** - * Saves the content of the `DataFrame` in ORC format at the specified path. - * - * @note Currently, this method can only be used after enabling Hive support - */ - def orc(path: String): Unit = { - format("orc").save(path) - } - - private[spark] def save(): Unit = { - graph.cypherSession.writeGraph(graph, config) - } - -} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 02e89909de530..b4f4d6e3e6105 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -1,11 +1,10 @@ package org.apache.spark.graph.cypher import org.apache.spark.graph.api._ -import org.apache.spark.graph.api.io.{ReaderConfig, WriterConfig} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter import org.apache.spark.graph.cypher.io.ReadWriteGraph._ -import org.apache.spark.sql.{DataFrame, SparkSession, functions} +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, functions} 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} @@ -86,27 +85,29 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema, this) } - override private[spark] def readGraph(config: ReaderConfig): PropertyGraph = { - val graphImporter = GraphImporter(sparkSession, config) + private val DEFAULT_FORMAT = "parquet" + + override def load(path: String): PropertyGraph = { + val graphImporter = GraphImporter(sparkSession, path, DEFAULT_FORMAT) createGraph(graphImporter.nodeFrames, graphImporter.relationshipFrames) } - override private[spark] def writeGraph(graph: PropertyGraph, config: WriterConfig): Unit = { + override private[spark] def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit = { val relationalGraph = toRelationalGraph(graph) - val graphDirectoryStructure = SparkGraphDirectoryStructure(config.path) + val graphDirectoryStructure = SparkGraphDirectoryStructure(path) relationalGraph.schema.labelCombinations.combos.foreach { combo => relationalGraph.canonicalNodeTable(combo) .write - .format(config.source) - .mode(config.mode) + .format(DEFAULT_FORMAT) + .mode(saveMode) .save(graphDirectoryStructure.pathToNodeTable(combo)) } relationalGraph.schema.relationshipTypes.foreach { relType => relationalGraph.canonicalRelationshipTable(relType) .write - .format(config.source) - .mode(config.mode) + .format(DEFAULT_FORMAT) + .mode(saveMode) .save(graphDirectoryStructure.pathToRelationshipTable(relType)) } } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala index 4cfa29b863718..8b680584711b3 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala @@ -3,7 +3,6 @@ package org.apache.spark.graph.cypher.io import java.net.URI import org.apache.hadoop.fs.FileSystem -import org.apache.spark.graph.api.io.ReaderConfig import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} import org.apache.spark.graph.cypher.SparkGraphDirectoryStructure import org.apache.spark.graph.cypher.SparkGraphDirectoryStructure._ @@ -17,11 +16,11 @@ import org.opencypher.okapi.relational.api.graph.RelationalCypherGraph object ReadWriteGraph { - case class GraphImporter(sparkSession: SparkSession, config: ReaderConfig) { + case class GraphImporter(sparkSession: SparkSession, path: String, format: String) { - val directoryStructure: SparkGraphDirectoryStructure = SparkGraphDirectoryStructure(config.path) + val directoryStructure: SparkGraphDirectoryStructure = SparkGraphDirectoryStructure(path) val (labelCombos, relTypes): (Seq[Set[String]], Seq[String]) = { - val fs = FileSystem.get(new URI(config.path), sparkSession.sparkContext.hadoopConfiguration) + val fs = FileSystem.get(new URI(path), sparkSession.sparkContext.hadoopConfiguration) try { import org.apache.spark.graph.cypher.util.HadoopFSUtils._ val combos = fs.listDirectories(directoryStructure.pathToNodeDirectory).map(_.toLabelCombo) @@ -34,7 +33,7 @@ object ReadWriteGraph { def nodeFrames: Seq[NodeFrame] = { labelCombos.map { combo => - val df = sparkSession.read.format(config.source).load(directoryStructure.pathToNodeTable(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 @@ -48,7 +47,7 @@ object ReadWriteGraph { def relationshipFrames: Seq[RelationshipFrame] = { relTypes.map { relType => - val df = sparkSession.read.format(config.source).load(directoryStructure.pathToRelationshipTable(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 diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala index cccc306cb97b8..9bc4248389f5b 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala @@ -42,21 +42,11 @@ class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS" ) - test("write a graph with orc") { + test("save and load a graph") { val graph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) - graph.write.orc(basePath) + graph.save(basePath) - val readGraph = cypherSession.read.orc(basePath) - readGraph.cypher( - "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" - ).df.show() - } - - test("write a graph with parquet") { - val graph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) - graph.write.parquet(basePath) - - val readGraph = cypherSession.read.parquet(basePath) + val readGraph = cypherSession.load(basePath) readGraph.cypher( "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" ).df.show() From d6f574f93804fc9719b92ae8657edf345ce1e73b Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 11:08:48 +0100 Subject: [PATCH 053/123] Remove result -> graph conversion Co-authored-by: Philip Stutz --- .../apache/spark/graph/api/CypherResult.scala | 45 --------- .../graph/cypher/SparkCypherResult.scala | 95 +------------------ .../graph/cypher/SparkCypherSession.scala | 3 +- .../graph/cypher/algos/GraphAlgorithms.scala | 50 ---------- .../spark/graph/cypher/BasicMatchSuite.scala | 48 +--------- 5 files changed, 6 insertions(+), 235 deletions(-) delete mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala 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 index dcd06ba04a8b8..80d7c9cd1fa3e 100644 --- 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 @@ -18,49 +18,4 @@ trait CypherResult { * (e.g. `MATCH (n:Person) RETURN n` results in the columns `n`, `n:Person` and `n_name`). */ def df: DataFrame - - // TODO: determine what happens if there are dangling relationships (create empty nodes or filter such relationships) - /** - * Creates a [[PropertyGraph]] from nodes and relationship present in this result. - * - * @example - * {{{ - * val result = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") - * // returns the original graph - * val graph2 = result.graph - * }}} - */ - def graph: PropertyGraph - - /** - * Extracts nodes that are specified as a return item. - * - * @example - * {{{ - * val result = graph.cypher("MATCH (n:Person)-[r:LIVES_IN]->(c:City) RETURN n, r, c") - * val nodeFrames = result.nodeFrames("n") - * }}} - * - * Returns a sequence of [[NodeFrame]]s where each single [[NodeFrame]] contains all distinct node entities that have at least label `:Person`. - * For example, if the graph contains nodes with label `:Person` and `:Person:Fireman` the sequence contains two [[NodeFrame]]s, one - * for each label combination. - * - * @param varName return item name of the node to extract - */ - def nodeFrames(varName: String): Seq[NodeFrame] - - /** - * Extracts relationships that are specified as a return item. - * - * @example - * {{{ - * val result = graph.cypher("MATCH (n:Person)-[r:LIVES_IN|WORKS_IN]->(c:City) RETURN n, r, c") - * val nodeFrames = result.nodeFrames("r") - * }}} - * - * Returns a sequence of two [[RelationshipFrame]]s, one that contains all distinct relationships with type `:LIVES_IN` and another for type `:WORKS_IN`. - * - * @param varName return item name of the relationships to extract - */ - def relationshipFrames(varName: String): Seq[RelationshipFrame] } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala index 3a2356763033f..2ca1442ee5156 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala @@ -1,99 +1,10 @@ package org.apache.spark.graph.cypher -import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.graph.api.CypherResult import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.sql.{DataFrame, functions} -import org.opencypher.okapi.api.schema.Schema -import org.opencypher.okapi.api.types.{CTNode, CTRelationship} -import org.opencypher.okapi.impl.exception.IllegalArgumentException -import org.opencypher.okapi.ir.api.expr._ +import org.apache.spark.sql.DataFrame import org.opencypher.okapi.relational.api.table.RelationalCypherRecords -import org.opencypher.okapi.relational.impl.table.RecordHeader - -case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable], schema: Schema, session: SparkCypherSession) extends CypherResult { +case class SparkCypherResult(relationalTable: RelationalCypherRecords[DataFrameTable]) extends CypherResult { override val df: DataFrame = relationalTable.table.df - - private val header: RecordHeader = relationalTable.header - - override def graph: PropertyGraph = { - val entityVars = relationalTable.header.entityVars - val nodeVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTNode) => v.name } - val relVarNames = entityVars.collect { case v if v.cypherType.subTypeOf(CTRelationship) => v.name } - - val nodeFrames = nodeVarNames.flatMap(this.nodeFrames).toSeq - val relFrames = relVarNames.flatMap(this.relationshipFrames).toSeq - - session.createGraph(nodeFrames, relFrames) - } - - // TODO: Error handling - override def nodeFrames(varName: String): Seq[NodeFrame] = { - val nodeVar: NodeVar = find(NodeVar(varName)(CTNode)) - - val idColumn = header.column(nodeVar) - val possibleLabels = header.labelsFor(nodeVar).map(_.label.name) - val labelCombinations = possibleLabels.flatMap(label => schema.combinationsFor(Set(label))) - - val labelToColumns = header.labelsFor(nodeVar).map(expr => expr.label.name -> header.column(expr)).toMap - val propertyToColumns = header.propertiesFor(nodeVar).map(expr => expr.key.name -> header.column(expr)).toMap - - val allLabelColumns = labelToColumns.values.toSet - - labelCombinations.toSeq.map { labels => - val trueLabels = labels.map(labelToColumns) - val falseLabels = allLabelColumns -- trueLabels - - val propertyKeys = schema.nodePropertyKeys(labels).keySet - val properties = propertyToColumns.filter { case (propertyKey, _) => propertyKeys.contains(propertyKey) } - val propertyColumns = properties.values.toSeq - - val selectColumns = (idColumn +: propertyColumns).map(escape) - val labelCombinationDf = df - .filter(trueLabels.map(df.col).map(_ === true).reduce(_ && _)) - .filter(falseLabels.map(df.col).map(_ === false).foldLeft(functions.lit(true))(_ && _)) - .select(selectColumns.head, selectColumns.tail: _*) - .dropDuplicates(idColumn) - - NodeFrame(labelCombinationDf, idColumn, labels, properties) - } - } - - // TODO: Error handling - override def relationshipFrames(varName: String): Seq[RelationshipFrame] = { - val relVar: RelationshipVar = find(RelationshipVar(varName)(CTRelationship)) - - val idColumn = header.column(relVar) - val relTypes = header.typesFor(relVar).map(_.relType.name).toSeq - val sourceIdColumn = header.column(header.startNodeFor(relVar)) - val targetIdColumn = header.column(header.endNodeFor(relVar)) - - val relTypeToColumns = header.typesFor(relVar).map(expr => expr.relType.name -> header.column(expr)).toMap - val propertyToColumns = header.propertiesFor(relVar).map(expr => expr.key.name -> header.column(expr)).toMap - - relTypes.map { relType => - val trueRelType = relTypeToColumns(relType) - val propertyKeys = schema.relationshipPropertyKeys(relType).keySet - val properties = propertyToColumns.filter { case (propertyKey, _) => propertyKeys.contains(propertyKey) } - val propertyColumns = properties.values.toSeq - - val selectColumns = (idColumn +: sourceIdColumn +: targetIdColumn +: propertyColumns).map(escape) - val relTypeDf = df - .filter(df.col(trueRelType) === true) - .select(selectColumns.head, selectColumns.tail: _*) - .dropDuplicates(idColumn) - - RelationshipFrame(relTypeDf, idColumn, sourceIdColumn, targetIdColumn, relType, properties) - } - } - - private def escape(columnName: String): String = s"`$columnName`" - - // TODO use header.entityVars ? - private def find[T <: ReturnItem](lookup: Var): T = relationalTable.header.idExpressions(lookup) - .collectFirst { case expr if expr.withoutType == lookup.withoutType => expr } - .getOrElse(throw IllegalArgumentException( - expected = s"One of ${relationalTable.header.vars.map(_.withoutType).toList}", - actual = lookup.name - )).asInstanceOf[T] } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index b4f4d6e3e6105..11fd2b77e65ed 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -58,6 +58,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) val trueLit = functions.lit(true) val falseLit = functions.lit(false) + // TODO: add empty set val nodeFrames = labelColumns.subsets().map { labelSet => val predicate = labelColumns.map { case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) === trueLit @@ -82,7 +83,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { val relationalGraph = toRelationalGraph(graph) - SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records, relationalGraph.schema, this) + SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records) } private val DEFAULT_FORMAT = "parquet" diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala deleted file mode 100644 index 24daac4431c40..0000000000000 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/algos/GraphAlgorithms.scala +++ /dev/null @@ -1,50 +0,0 @@ -package org.apache.spark.graph.cypher.algos - -import org.apache.spark.graph.api.PropertyGraph -import org.apache.spark.graphx.{Edge, Graph} -import org.apache.spark.sql.types.BinaryType - -object GraphAlgorithms { - - implicit class PageRankGraph(graph: PropertyGraph) { - - def withPageRank(tolerance: Double = 0.01, resetProbability: Double = 0.15): PropertyGraph = { - // Create GraphX compatible RDDs from nodes and relationships - val graphXNodeRDD = graph - .nodes - .select("$ID") - .rdd.map(row => row.getAs[Array[Byte]](0).toLong -> null) - - val graphXRelRDD = graph - .relationships - .select("$SOURCE_ID", "$TARGET_ID") - .rdd.map(row => Edge(row.getAs[Array[Byte]](0).toLong, row.getAs[Array[Byte]](1).toLong, ())) - - // Compute Page Rank via GraphX - val graphX = Graph(graphXNodeRDD, graphXRelRDD) - val ranks = graphX.pageRank(tolerance, resetProbability).vertices - - // Convert RDD to DataFrame - val rankDf = graph.cypherSession.sparkSession.createDataFrame(ranks) - val rankTable = rankDf.select( - rankDf.col("_1").cast(BinaryType).as("$ID"), - rankDf.col("_2").as("pageRank")) - - graph.cypherSession.createGraph(graph.nodes.join(rankTable, "$ID"), graph.relationships) - } - } - - implicit class ByteArrayConversion(val a: Array[Byte]) extends AnyVal { - def toLong: Long = { - var result = a(0).toLong << 56 - result |= a(1).toLong << 48 - result |= a(2).toLong << 40 - result |= a(3).toLong << 32 - result |= a(4).toLong << 24 - result |= a(5).toLong << 16 - result |= a(6).toLong << 8 - result |= a(7).toLong - result - } - } -} diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala index 63b56e4904d79..2e017c2c0981c 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala @@ -1,8 +1,7 @@ package org.apache.spark.graph.cypher import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} -import org.apache.spark.graph.cypher.algos.GraphAlgorithms._ +import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} import org.apache.spark.sql.DataFrame class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { @@ -29,38 +28,6 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() } - test("create property graph from query results") { - val personData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, "Alice", 42), Tuple3(1, "Bob", 23), Tuple3(2, "Eve", 19))).toDF("id", "name", "age") - val universityData: DataFrame = spark.createDataFrame(Seq(2 -> "UC Berkeley", 3 -> "Stanford")).toDF("id", "title") - val knowsData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, 0, 1), Tuple3(1, 0, 2))).toDF("id", "source", "target") - val studyAtData: DataFrame = spark.createDataFrame(Seq(Tuple3(2, 0, 2), Tuple3(3, 1, 3), Tuple3(4, 2, 2))).toDF("id", "source", "target") - val personDataFrame: NodeFrame = NodeFrame(df = personData, idColumn = "id", labels = Set("Student")) - val universityDataFrame: NodeFrame = NodeFrame(df = universityData, idColumn = "id", labels = Set("University")) - val knowsDataFrame: RelationshipFrame = RelationshipFrame(knowsData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val studyAtDataFrame: RelationshipFrame = RelationshipFrame(studyAtData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "STUDY_AT") - - val graph: PropertyGraph = cypherSession.createGraph(Seq(personDataFrame, universityDataFrame), Seq(knowsDataFrame, studyAtDataFrame)) - - val result: CypherResult = graph.cypher( - """ - |MATCH (p:Student)-[:STUDY_AT]->(u:University), - | (p)-[k:KNOWS]->(o:Student)-[:STUDY_AT]->(u) - |WHERE u.title = 'UC Berkeley' - |RETURN p, o, k - |""".stripMargin) - - // Option 1: Return NodeFrames and RelationshipFrames - val berkeleyStudents: Seq[NodeFrame] = result.nodeFrames("p") - val berkeleyStudentFriends: Seq[NodeFrame] = result.nodeFrames("o") - val knows: Seq[RelationshipFrame] = result.relationshipFrames("k") - val berkeleyGraph: PropertyGraph = cypherSession.createGraph(berkeleyStudents ++ berkeleyStudentFriends, knows) - berkeleyGraph.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.show() - - // Option 2: Use CypherResult to create a new PropertyGraph - val berkeleyGraph2 = result.graph - berkeleyGraph2.cypher("MATCH (n:Student)-[:KNOWS]->(o:Student) RETURN n.name AS person, o.name AS friend").df.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) @@ -77,19 +44,6 @@ class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { .cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() } - test("query composition + graph analytics workflow") { - cypherSession.createGraph(nodes, relationships) - .cypher( - """|MATCH (student:Student)-[:STUDY_AT]->(uni:University), - | (student)-[knows:KNOWS]-(:Student) - |WHERE uni.title = 'UC Berkeley' - |RETURN student, knows""".stripMargin) - .graph - .withPageRank(tolerance = 0.02) - .cypher("MATCH (n:Student) RETURN n.name, n.pageRank ORDER BY n.pageRank DESC") - .df.show() - } - lazy val nodes: DataFrame = spark.createDataFrame(Seq( (0L, true, false, Some("Alice"), Some(42), None), (1L, true, false, Some("Bob"), Some(23), None), From 200d957fd17591423199400307051d2077742ac7 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 11:56:04 +0100 Subject: [PATCH 054/123] Adapt implementation to okapi-relational 0.3.1-SNAPSHOT Co-authored-by: Philip Stutz --- .../graph/cypher/SparkCypherSession.scala | 1 - .../spark/graph/cypher/SparkTable.scala | 83 +-- .../cypher/conversions/ExprConversions.scala | 542 +++++++----------- .../conversions/TemporalConversions.scala | 2 +- .../cypher/conversions/TypeConversions.scala | 35 +- .../spark/graph/cypher/udfs/LegacyUdfs.scala | 29 - .../cypher/udfs/SparkCypherFunctions.scala | 160 ++++++ .../graph/cypher/SharedCypherContext.scala | 8 +- pom.xml | 2 + 9 files changed, 434 insertions(+), 428 deletions(-) delete mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 11fd2b77e65ed..933b9407e06ad 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -27,7 +27,6 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) override type Result = RelationalCypherResult[DataFrameTable] override type Records = SparkCypherRecords - override type Graph = RelationalCypherGraph[DataFrameTable] implicit def sparkCypherSession: SparkCypherSession = this diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala index e811e943c2359..1090bd24da94b 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala @@ -8,7 +8,7 @@ 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, NotImplementedException} +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._ @@ -38,7 +38,7 @@ object SparkTable { 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) }: _*) + df.select(columns.map { case (colName, alias) => df.col(s"`$colName`").as(alias) }: _*) } } @@ -99,7 +99,7 @@ object SparkTable { df.limit(items.toInt) } - override def group(by: Set[Var], aggregations: Set[(Aggregator, (String, CypherType))]) + override def group(by: Set[Var], aggregations: Map[String, Aggregator]) (implicit header: RecordHeader, parameters: CypherMap): DataFrameTable = { def withInnerExpr(expr: Expr)(f: Column => Column) = @@ -117,49 +117,7 @@ object SparkTable { } val sparkAggFunctions = aggregations.map { - case (aggFunc, (columnName, cypherType)) => - aggFunc match { - case Avg(expr) => - withInnerExpr(expr)( - functions - .avg(_) - .cast(cypherType.getSparkType) - .as(columnName)) - - case CountStar(_) => - functions.count(functions.lit(0)).as(columnName) - - // TODO: Consider not implicitly projecting the aggFunc expr here, but rewriting it into a variable in logical planning or IR construction - case Count(expr, distinct) => withInnerExpr(expr) { column => - val count = { - if (distinct) functions.countDistinct(column) - else functions.count(column) - } - count.as(columnName) - } - - case Max(expr) => - withInnerExpr(expr)(functions.max(_).as(columnName)) - - case Min(expr) => - withInnerExpr(expr)(functions.min(_).as(columnName)) - - case Sum(expr) => - withInnerExpr(expr)(functions.sum(_).as(columnName)) - - case Collect(expr, distinct) => withInnerExpr(expr) { column => - val list = { - if (distinct) functions.collect_set(column) - else functions.collect_list(column) - } - // sort for deterministic aggregation results - val sorted = functions.sort_array(list) - sorted.as(columnName) - } - - case x => - throw NotImplementedException(s"Aggregation function $x") - } + case (columnName, aggFunc) => aggFunc.asSparkSQLExpr(header, df, parameters).as(columnName) } data.fold( @@ -193,8 +151,15 @@ object SparkTable { } joinType match { - case CrossJoin => df.crossJoin(other.df) - case _ => df.safeJoin(other.df, joinCols, joinTypeString) + 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) } } @@ -229,10 +194,13 @@ object SparkTable { require(joinCols.map(_._1).forall(col => !other.columns.contains(col))) require(joinCols.map(_._2).forall(col => !df.columns.contains(col))) - val joinExpr = joinCols.map { - case (l, r) => df.col(l) === other.col(r) - }.reduce((acc, expr) => acc && expr) - + 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) } @@ -257,16 +225,19 @@ object SparkTable { df.schema.fields(df.schema.fieldIndex(columnName)) } + def safeRenameColumns(renames: (String, String)*): DataFrame = { + safeRenameColumns(renames.toMap) + } + def safeRenameColumns(renames: Map[String, String]): DataFrame = { - val actualRenames = renames.filter { case (oldCol, newCol) => oldCol != newCol } - if (actualRenames.isEmpty) { + if (renames.isEmpty || renames.forall { case (oldColumn, newColumn) => oldColumn == newColumn }) { df } else { - actualRenames.foreach { case (oldName, newName) => require(!df.columns.contains(newName), + 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 actualRenames.contains(col) => actualRenames(col) + 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/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala index a9a155d81e756..c26eb1dd79845 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -1,15 +1,14 @@ package org.apache.spark.graph.cypher.conversions +import org.apache.spark.sql.functions.{array_contains => _, translate => _, _} import org.apache.spark.graph.cypher.conversions.TemporalConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ -import org.apache.spark.graph.cypher.udfs.LegacyUdfs._ +import org.apache.spark.graph.cypher.udfs.SparkCypherFunctions._ import org.apache.spark.graph.cypher.udfs.TemporalUdfs -import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue -import org.apache.spark.sql.catalyst.expressions.{ArrayContains, StringTranslate} import org.apache.spark.sql.types._ -import org.apache.spark.sql.{Column, DataFrame, functions} +import org.apache.spark.sql.{Column, DataFrame} import org.opencypher.okapi.api.types._ -import org.opencypher.okapi.api.value.CypherValue.{CypherList, CypherMap} +import org.opencypher.okapi.api.value.CypherValue.CypherMap import org.opencypher.okapi.impl.exception._ import org.opencypher.okapi.impl.temporal.TemporalTypesHelper._ import org.opencypher.okapi.impl.temporal.{Duration => DurationValue} @@ -19,40 +18,8 @@ import org.opencypher.okapi.relational.impl.table.RecordHeader object ExprConversions { - final case class ExprConversionException(msg: String) extends InternalException(msg) - - private val NULL_LIT: Column = functions.lit(null) - - private val TRUE_LIT: Column = functions.lit(true) - - private val FALSE_LIT: Column = functions.lit(false) - - private val ONE_LIT: Column = functions.lit(1) - - private val E: Column = functions.lit(Math.E) - - private val PI: Column = functions.lit(Math.PI) - implicit class RichExpression(expr: Expr) { - /** - * This is possible without violating Cypher semantics because - * - Spark SQL returns null when comparing across types (from initial investigation) - * - We never have multiple types per column in CAPS (yet) - */ - def compare(comparator: Column => Column => Column, lhs: Expr, rhs: Expr) - (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { - comparator(lhs.asSparkSQLExpr)(rhs.asSparkSQLExpr) - } - - def lt(column: Column): Column => Column = column < _ - - def lteq(column: Column): Column => Column = column <= _ - - def gt(column: Column): Column => Column = column > _ - - def gteq(column: Column): Column => Column = column >= _ - /** * Attempts to create a Spark SQL expression from the CAPS expression. * @@ -62,66 +29,79 @@ object ExprConversions { * @return Some Spark SQL expression if the input was mappable, otherwise None. */ def asSparkSQLExpr(implicit header: RecordHeader, df: DataFrame, 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 _ => null_safe_conversion(expr)(convert) + } + header.getColumn(expr) match { + case None => outCol + case Some(colName) => outCol.as(colName) + } + } - expr match { + private def convert(convertedChildren: Seq[Column]) + (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { - case AliasExpr(innerExpr, _) => innerExpr.asSparkSQLExpr + def child0: Column = convertedChildren.head - case Explode(list) => list.cypherType match { - case CTList(_) | CTListOrNull(_) => functions.explode(list.asSparkSQLExpr) - case CTNull => functions.explode(functions.lit(null).cast(ArrayType(NullType))) - case other => throw IllegalArgumentException("CTList", other) - } + def child1: Column = convertedChildren(1) - case e if e.cypherType == CTNull => NULL_LIT + def child2: Column = convertedChildren(2) - case Param(name) => - expr.cypherType match { - case CTList(inner) => - if (inner == CTAny) { - throw ExprConversionException(s"List parameter with inner type $inner not supported") - } else { - functions.array(parameters(name).asInstanceOf[CypherList].value.unwrap.map(functions.lit): _*) - } - case _ => toSparkLiteral(parameters(name).unwrap) - } - - case ListLit(exprs) => - if (expr.cypherType == CTAny) { - throw ExprConversionException(s"List literal with inner type ${expr.cypherType} not supported") - } else { - functions.array(exprs.map(_.asSparkSQLExpr): _*) + 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 Property(e, PropertyKey(key)) => + // TODO: Convert property lookups into separate specific lookups instead of overloading e.cypherType.material match { - case CTMap(inner) => - if (inner.keySet.contains(key)) e.asSparkSQLExpr.getField(key) else functions.lit(null) - - case CTDate => - temporalAccessor[Date](e.asSparkSQLExpr, key) - - case CTLocalDateTime => - temporalAccessor[Timestamp](e.asSparkSQLExpr, key) - - case CTDuration => - TemporalUdfs.durationAccessor(key.toLowerCase).apply(e.asSparkSQLExpr) - - case _ if !header.contains(expr) || !df.columns.contains(header.column(expr)) => - NULL_LIT - + case CTMap(inner) => if (inner.keySet.contains(key)) child0.getField(key) else NULL_LIT + case CTDate => temporalAccessor[java.sql.Date](child0, key) + case CTLocalDateTime => temporalAccessor[java.sql.Timestamp](child0, key) + case CTDuration => TemporalUdfs.durationAccessor(key.toLowerCase).apply(child0) case _ => - columnFor(expr) + if (!header.contains(expr)) { + NULL_LIT + } else { + column_for(expr) + } } - - case IsNull(e) => e.asSparkSQLExpr.isNull - case IsNotNull(e) => e.asSparkSQLExpr.isNotNull - - case _: Var | _: Param | _: HasLabel | _: HasType | _: StartNode | _: EndNode => columnFor(expr) - - case NullLit(ct) => NULL_LIT.cast(ct.getSparkType) - case LocalDateTime(dateExpr) => + // TODO: Move code outside of expr mapper dateExpr match { case Some(e) => val localDateTimeValue = resolveTemporalArgument(e) @@ -133,11 +113,12 @@ object ExprConversions { } .orNull - functions.lit(localDateTimeValue).cast(DataTypes.TimestampType) - case None => functions.current_timestamp() + lit(localDateTimeValue).cast(DataTypes.TimestampType) + case None => current_timestamp() } case Date(dateExpr) => + // TODO: Move code outside of expr mapper dateExpr match { case Some(e) => val dateValue = resolveTemporalArgument(e) @@ -145,325 +126,218 @@ object ExprConversions { .map(java.sql.Date.valueOf) .orNull - functions.lit(dateValue).cast(DataTypes.DateType) - case None => functions.current_timestamp() + lit(dateValue).cast(DataTypes.DateType) + case None => current_timestamp() } case Duration(durationExpr) => + // TODO: Move code outside of expr mapper val durationValue = resolveTemporalArgument(durationExpr).map { case Left(m) => DurationValue(m.mapValues(_.toLong)).toCalendarInterval case Right(s) => DurationValue.parse(s).toCalendarInterval }.orNull - functions.lit(durationValue) - - case l: Lit[_] => functions.lit(l.v) + lit(durationValue) - // predicates - case Equals(e1, e2) => e1.asSparkSQLExpr === e2.asSparkSQLExpr - case Not(e) => !e.asSparkSQLExpr - case Size(e) => - val col = e.asSparkSQLExpr - e.cypherType match { - case CTString => functions.length(col).cast(LongType) - case _: CTList | _: CTListOrNull => - functions.when( - col.isNotNull, - functions.size(col).cast(LongType) - ) - case other => throw NotImplementedException(s"size() on values of type $other") - } - - case Ands(exprs) => exprs.map(_.asSparkSQLExpr).foldLeft(TRUE_LIT)(_ && _) - case Ors(exprs) => exprs.map(_.asSparkSQLExpr).foldLeft(FALSE_LIT)(_ || _) - - case In(lhs, rhs) if lhs.cypherType == CTNull => - val array = rhs.asSparkSQLExpr - functions - .when(functions.size(array) === 0, FALSE_LIT) - .otherwise(NULL_LIT) - - case In(lhs, rhs) => - val element = lhs.asSparkSQLExpr - val array = rhs.asSparkSQLExpr - functions - .when(functions.size(array) === 0, FALSE_LIT) - .when(array.isNull, NULL_LIT) - .when(element.isNull, NULL_LIT) - .otherwise(new Column(ArrayContains(array.expr, element.expr))) - - case LessThan(lhs, rhs) => compare(lt, lhs, rhs) - case LessThanOrEqual(lhs, rhs) => compare(lteq, lhs, rhs) - case GreaterThanOrEqual(lhs, rhs) => compare(gteq, lhs, rhs) - case GreaterThan(lhs, rhs) => compare(gt, lhs, rhs) - - case StartsWith(lhs, rhs) => lhs.asSparkSQLExpr.startsWith(rhs.asSparkSQLExpr) - case EndsWith(lhs, rhs) => lhs.asSparkSQLExpr.endsWith(rhs.asSparkSQLExpr) - case Contains(lhs, rhs) => lhs.asSparkSQLExpr.contains(rhs.asSparkSQLExpr) - - case RegexMatch(prop, Param(name)) => - val regex: String = parameters(name).unwrap.toString - prop.asSparkSQLExpr.rlike(regex) + case In(lhs, rhs) => rhs.cypherType.material match { + case CTList(CTVoid) => FALSE_LIT + case CTList(inner) if inner.couldBeSameTypeAs(lhs.cypherType) => array_contains(child1, child0) + case _ => NULL_LIT + } // Arithmetic case Add(lhs, rhs) => - val lhsCT = lhs.cypherType - val rhsCT = rhs.cypherType - if (rhsCT == CTNull || rhsCT == CTNull) { - NULL_LIT - } else { - lhsCT.material -> rhsCT.material match { - case (CTList(lhInner), CTList(rhInner)) => - if (lhInner.material == rhInner.material || lhInner == CTVoid || rhInner == CTVoid) { - functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) - } else { - throw ExprConversionException(s"Lists of different inner types are not supported (${lhInner.material}, ${rhInner.material})") - } - - case (CTList(inner), nonListType) if nonListType == inner.material || inner.material == CTVoid => - functions.concat(lhs.asSparkSQLExpr, functions.array(rhs.asSparkSQLExpr)) - - case (nonListType, CTList(inner)) if inner.material == nonListType || inner.material == CTVoid => - functions.concat(functions.array(lhs.asSparkSQLExpr), rhs.asSparkSQLExpr) - - case (CTString, _) if rhsCT.subTypeOf(CTNumber) => - functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr.cast(StringType)) - - case (_, CTString) if lhsCT.subTypeOf(CTNumber) => - functions.concat(lhs.asSparkSQLExpr.cast(StringType), rhs.asSparkSQLExpr) - - case (CTString, CTString) => - functions.concat(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) - - case (CTDate, CTDuration) => - TemporalUdfs.dateAdd(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) - - case _ => - lhs.asSparkSQLExpr + rhs.asSparkSQLExpr - } + val lhsCT = lhs.cypherType.material + val rhsCT = rhs.cypherType.material + lhsCT -> rhsCT match { + case (CTList(lhInner), CTList(rhInner)) => + if (lhInner.material == rhInner.material || lhInner == CTVoid || rhInner == CTVoid) { + concat(child0, child1) + } else { + throw NotImplementedException(s"Lists of different inner types are not supported (${lhInner.material}, ${rhInner.material})") + } + case (CTList(inner), nonListType) if nonListType == inner.material || inner.material == CTVoid => concat(child0, array(child1)) + case (nonListType, CTList(inner)) if inner.material == nonListType || inner.material == CTVoid => 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(lhs.asSparkSQLExpr, rhs.asSparkSQLExpr) + TemporalUdfs.dateSubtract(child0, child1) - case Subtract(lhs, rhs) => lhs.asSparkSQLExpr - rhs.asSparkSQLExpr + case _: Subtract => child0 - child1 - case Multiply(lhs, rhs) => lhs.asSparkSQLExpr * rhs.asSparkSQLExpr - case div@Divide(lhs, rhs) => (lhs.asSparkSQLExpr / rhs.asSparkSQLExpr).cast(div.cypherType.getSparkType) + case _: Multiply => child0 * child1 + case div: Divide => (child0 / child1).cast(div.cypherType.getSparkType) // Id functions - - case Id(e) => e.asSparkSQLExpr + case _: Id => child0 // Functions - case _: MonotonicallyIncreasingId => functions.monotonically_increasing_id() - case Exists(e) => e.asSparkSQLExpr.isNotNull + case _: MonotonicallyIncreasingId => monotonically_increasing_id() case Labels(e) => - e.cypherType match { - case _: CTNode | _: CTNodeOrNull => - val node = e.owner.get - val labelExprs = header.labelsFor(node) - val (labelNames, labelColumns) = labelExprs - .toSeq - .map(e => e.label.name -> e.asSparkSQLExpr) - .sortBy(_._1) - .unzip - val booleanLabelFlagColumn = functions.array(labelColumns: _*) - get_node_labels(labelNames)(booleanLabelFlagColumn) - case other => throw IllegalArgumentException("an expression with type CTNode, CTNodeOrNull, or CTNull", other) - } + 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 _: CTNode | _: CTRelationship => + 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 Keys(e) => e.cypherType.material match { - case _: CTNode | _: CTRelationship => - val node = e.owner.get - val propertyExprs = header.propertiesFor(node).toSeq.sortBy(_.key.name) - val (propertyKeys, propertyColumns) = propertyExprs.map(e => e.key.name -> e.asSparkSQLExpr).unzip - val valuesColumn = functions.array(propertyColumns: _*) - get_property_keys(propertyKeys)(valuesColumn) - - case CTMap(innerTypes) => - val mapColumn = e.asSparkSQLExpr - val (keys, valueColumns) = innerTypes.keys.map { e => - // Whe have to make sure that every column has the same type (true or null) - e -> functions.when(mapColumn.getField(e).isNotNull, functions.lit(true)).otherwise(NULL_LIT) - }.toSeq.unzip - val valueColumn = functions.array(valueColumns: _*) - get_property_keys(keys)(valueColumn) - - case other => throw IllegalArgumentException("an Expression with type CTNode, CTRelationship or CTMap", other) - } + 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 _: CTNode | _: CTRelationship => - val element = e.owner.get - val propertyExprs = header.propertiesFor(element).toSeq.sortBy(_.key.name) - val propertyColumns = propertyExprs.map(e => e.asSparkSQLExpr.as(e.key.name)) - createStructColumn(propertyColumns) - case _: CTMap => e.asSparkSQLExpr + 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 Type(inner) => - inner match { - case v: Var => - val typeExprs = header.typesFor(v) - val (relTypeNames, relTypeColumn) = typeExprs.toSeq.map(e => e.relType.name -> e.asSparkSQLExpr).unzip - val booleanLabelFlagColumn = functions.array(relTypeColumn: _*) - get_rel_type(relTypeNames)(booleanLabelFlagColumn) - case _ => - throw NotImplementedException(s"Inner expression $inner of $expr is not yet supported (only variables)") - } - - case StartNodeFunction(e) => - val rel = e.owner.get - header.startNodeFor(rel).asSparkSQLExpr - - case EndNodeFunction(e) => - val rel = e.owner.get - header.endNodeFor(rel).asSparkSQLExpr + case StartNodeFunction(e) => header.startNodeFor(e.owner.get).asSparkSQLExpr + case EndNodeFunction(e) => header.endNodeFor(e.owner.get).asSparkSQLExpr - case ToFloat(e) => e.asSparkSQLExpr.cast(DoubleType) - case ToInteger(e) => e.asSparkSQLExpr.cast(IntegerType) - case ToString(e) => e.asSparkSQLExpr.cast(StringType) - case ToBoolean(e) => e.asSparkSQLExpr.cast(BooleanType) + case _: ToFloat => child0.cast(DoubleType) + case _: ToInteger => child0.cast(IntegerType) + case _: ToString => child0.cast(StringType) + case _: ToBoolean => child0.cast(BooleanType) - case Trim(str) => functions.trim(str.asSparkSQLExpr) - case LTrim(str) => functions.ltrim(str.asSparkSQLExpr) - case RTrim(str) => functions.rtrim(str.asSparkSQLExpr) + case _: Trim => trim(child0) + case _: LTrim => ltrim(child0) + case _: RTrim => rtrim(child0) + case _: ToUpper => upper(child0) + case _: ToLower => lower(child0) - case ToUpper(str) => functions.upper(str.asSparkSQLExpr) - case ToLower(str) => functions.lower(str.asSparkSQLExpr) + case _: Range => sequence(child0, child1, convertedChildren.lift(2).getOrElse(ONE_LIT)) - case Range(from, to, maybeStep) => - val stepCol = maybeStep.map(_.asSparkSQLExpr).getOrElse(ONE_LIT) - functions.sequence(from.asSparkSQLExpr, to.asSparkSQLExpr, stepCol) + case _: Replace => translate(child0, child1, child2) - case Replace(original, search, replacement) => - new Column(StringTranslate(original.asSparkSQLExpr.expr, search.asSparkSQLExpr.expr, replacement.asSparkSQLExpr.expr)) - case Substring(original, start, maybeLength) => - val origCol = original.asSparkSQLExpr - val startCol = start.asSparkSQLExpr + ONE_LIT - val lengthCol = maybeLength.map(_.asSparkSQLExpr).getOrElse(functions.length(origCol) - startCol + ONE_LIT) - origCol.substr(startCol, lengthCol) + case _: Substring => child0.substr(child1 + ONE_LIT, convertedChildren.lift(2).getOrElse(length(child0) - child1)) // Mathematical functions - - case _: E => E - case _: Pi => PI - - case Sqrt(e) => functions.sqrt(e.asSparkSQLExpr) - case Log(e) => functions.log(e.asSparkSQLExpr) - case Log10(e) => functions.log(10.0, e.asSparkSQLExpr) - case Exp(e) => functions.exp(e.asSparkSQLExpr) - case Abs(e) => functions.abs(e.asSparkSQLExpr) - case Ceil(e) => functions.ceil(e.asSparkSQLExpr).cast(DoubleType) - case Floor(e) => functions.floor(e.asSparkSQLExpr).cast(DoubleType) - case _: Rand => functions.rand() - case Round(e) => functions.round(e.asSparkSQLExpr).cast(DoubleType) - case Sign(e) => functions.signum(e.asSparkSQLExpr).cast(IntegerType) - - case Acos(e) => functions.acos(e.asSparkSQLExpr) - case Asin(e) => functions.asin(e.asSparkSQLExpr) - case Atan(e) => functions.atan(e.asSparkSQLExpr) - case Atan2(e1, e2) => functions.atan2(e1.asSparkSQLExpr, e2.asSparkSQLExpr) - case Cos(e) => functions.cos(e.asSparkSQLExpr) - case Cot(e) => Divide(IntegerLit(1)(CTInteger), Tan(e)(CTFloat))(CTFloat).asSparkSQLExpr - case Degrees(e) => functions.degrees(e.asSparkSQLExpr) - case Haversin(e) => Divide(Subtract(IntegerLit(1)(CTInteger), Cos(e)(CTFloat))(CTFloat), IntegerLit(2)(CTInteger))(CTFloat).asSparkSQLExpr - case Radians(e) => functions.radians(e.asSparkSQLExpr) - case Sin(e) => functions.sin(e.asSparkSQLExpr) - case Tan(e) => functions.tan(e.asSparkSQLExpr) + 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))(CTFloat).asSparkSQLExpr + case _: Degrees => degrees(child0) + case Haversin(e) => Divide(Subtract(IntegerLit(1), Cos(e))(CTFloat), IntegerLit(2))(CTFloat).asSparkSQLExpr + case _: Radians => radians(child0) + case _: Sin => sin(child0) + case _: Tan => tan(child0) // Time functions - - case Timestamp() => functions.current_timestamp().cast(LongType) + case Timestamp => current_timestamp().cast(LongType) // Bit operations - - case BitwiseAnd(lhs, rhs) => lhs.asSparkSQLExpr.bitwiseAND(rhs.asSparkSQLExpr) - case BitwiseOr(lhs, rhs) => lhs.asSparkSQLExpr.bitwiseOR(rhs.asSparkSQLExpr) - case ShiftLeft(value, IntegerLit(shiftBits)) => functions.shiftLeft(value.asSparkSQLExpr, shiftBits.toInt) - case ShiftRightUnsigned(value, IntegerLit(shiftBits)) => functions.shiftRightUnsigned(value.asSparkSQLExpr, shiftBits.toInt) + 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) - functions.coalesce(columns: _*) - - case c: CaseExpr => - val alternatives = c.alternatives.map { - case (predicate, action) => functions.when(predicate.asSparkSQLExpr, action.asSparkSQLExpr) - } - - val alternativesWithDefault = c.default match { - case Some(inner) => alternatives :+ inner.asSparkSQLExpr - case None => alternatives - } + coalesce(columns: _*) - val reversedColumns = alternativesWithDefault.reverse - - val caseColumn = reversedColumns.tail.foldLeft(reversedColumns.head) { - case (tmpCol, whenCol) => whenCol.otherwise(tmpCol) + case CaseExpr(_, maybeDefault) => + val (maybeConvertedDefault, convertedAlternatives) = if (maybeDefault.isDefined) { + Some(convertedChildren.head) -> convertedChildren.tail + } else { + None -> convertedChildren } - caseColumn + 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 indexCol = index.asSparkSQLExpr val containerCol = container.asSparkSQLExpr container.cypherType.material match { - case _: CTList | _: CTMap => new Column(UnresolvedExtractValue(containerCol.expr, indexCol.expr)) + case _: CTList | _: CTMap => containerCol.get(indexCol) 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 - createStructColumn(innerColumns) + 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 _ => throw NotImplementedException(s"No support for converting Cypher expression $expr to a Spark SQL expression") } } - } - - private def columnFor(expr: Expr)(implicit header: RecordHeader, df: DataFrame): Column = { - val columnName = header.getColumn(expr).getOrElse(throw IllegalArgumentException( - expected = s"Expression in ${header.expressions.mkString("[", ", ", "]")}", - actual = expr) - ) - if (df.columns.contains(columnName)) { - df.col(columnName) - } else { - NULL_LIT - } - } - private def toSparkLiteral(value: Any): Column = value match { - case map: Map[_, _] => - val columns = map.map { - case (key, v) => toSparkLiteral(v).as(key.toString) - }.toSeq - createStructColumn(columns) - case _ => functions.lit(value) } - - private def createStructColumn(structColumns: Seq[Column]): Column = { - if (structColumns.isEmpty) { - functions.lit(null).cast(new StructType()) - } else { - functions.struct(structColumns: _*) - } - } - - } diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala index 310a4c92e0d73..8669374061be6 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala @@ -85,7 +85,7 @@ object TemporalConversions { Some(Right(s)) - case NullLit(_) => None + case NullLit => None case other => throw NotImplementedException(s"Parsing temporal values is currently only supported for Literal-Maps and String literals, got $other") diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala index 994e42a46f387..372a706b09742 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala @@ -1,9 +1,12 @@ package org.apache.spark.graph.cypher.conversions +import org.apache.spark.graph.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 @@ -37,7 +40,7 @@ object TypeConversions { } def toSparkType: Option[DataType] = ct match { - case CTNull | CTVoid => Some(NullType) + case CTNull => Some(NullType) case _ => ct.material match { case CTString => Some(StringType) @@ -50,10 +53,12 @@ object TypeConversions { case CTIdentity => Some(BinaryType) case _: CTNode => Some(BinaryType) case _: CTRelationship => Some(BinaryType) - case CTList(CTVoid) => Some(ArrayType(NullType, containsNull = true)) - case CTList(CTNull) => Some(ArrayType(NullType, containsNull = true)) - case CTList(elemType) => - elemType.toSparkType.map(ArrayType(_, elemType.isNullable)) + // Spark uses String as the default array inner type + case CTList(CTVoid) => Some(ArrayType(StringType, containsNull = false)) + case CTList(CTNull) => Some(ArrayType(StringType, containsNull = true)) + case CTList(CTNumber) => Some(ArrayType(DoubleType, containsNull = false)) + case CTList(CTNumber.nullable) => Some(ArrayType(DoubleType, containsNull = true)) + case CTList(elemType) => elemType.toSparkType.map(ArrayType(_, elemType.isNullable)) case CTMap(inner) => val innerFields = inner.map { case (key, valueType) => valueType.toStructField(key) @@ -66,11 +71,13 @@ object TypeConversions { def getSparkType: DataType = toSparkType match { case Some(t) => t - case None => throw NotImplementedException(s"Mapping of CypherType $ct to Spark type") + 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) { @@ -178,5 +185,21 @@ object TypeConversions { 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/graph/cypher/udfs/LegacyUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala deleted file mode 100644 index 9bd5d6315d791..0000000000000 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/LegacyUdfs.scala +++ /dev/null @@ -1,29 +0,0 @@ -package org.apache.spark.graph.cypher.udfs - -import org.apache.spark.sql.expressions.UserDefinedFunction -import org.apache.spark.sql.functions -import org.apache.spark.sql.types.{ArrayType, StringType} - -object LegacyUdfs { - - def get_rel_type(relTypeNames: Seq[String]): UserDefinedFunction = { - val extractRelTypes = (booleanMask: Seq[Boolean]) => filterWithMask(relTypeNames)(booleanMask) - functions.udf(extractRelTypes.andThen(_.headOption.orNull), StringType) - } - - def get_node_labels(labelNames: Seq[String]): UserDefinedFunction = - functions.udf(filterWithMask(labelNames) _, ArrayType(StringType, containsNull = false)) - - private def filterWithMask(dataToFilter: Seq[String])(mask: Seq[Boolean]): Seq[String] = - dataToFilter.zip(mask).collect { - case (label, true) => label - } - - def get_property_keys(propertyKeys: Seq[String]): UserDefinedFunction = - functions.udf(filterNotNull(propertyKeys) _, ArrayType(StringType, containsNull = false)) - - private def filterNotNull(dataToFilter: Seq[String])(values: Seq[Any]): Seq[String] = - dataToFilter.zip(values).collect { - case (key, value) if value != null => key - } -} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala new file mode 100644 index 0000000000000..a3e8ebcb4b2c2 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala @@ -0,0 +1,160 @@ +package org.apache.spark.graph.cypher.udfs + +import org.apache.spark.graph.cypher.conversions.ExprConversions._ +import org.apache.spark.graph.cypher.conversions.TypeConversions._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.expressions.{ArrayContains, ArrayFilter, ArrayTransform, CaseWhen, EqualTo, Expression, GenericRowWithSchema, GetArrayItem, GetStructField, IsNotNull, LambdaFunction, NamedLambdaVariable, RLike, Slice, StringTranslate, XxHash64} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} +import org.apache.spark.sql.{Column, DataFrame, functions} +import org.opencypher.okapi.api.types.CTNull +import org.opencypher.okapi.api.types.CypherType._ +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))) + } + + 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))) + } + + 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))) + 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)) + } + + /** + * 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 null_safe_conversion(expr: Expr)(withConvertedChildren: Seq[Column] => Column) + (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { + if (expr.cypherType == CTNull) { + NULL_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) + } + } + } + + def column_for(expr: Expr)(implicit header: RecordHeader, df: DataFrame): Column = { + val columnName = header.getColumn(expr).getOrElse(throw IllegalArgumentException( + expected = s"Expression in ${header.expressions.mkString("[", ", ", "]")}", + actual = expr) + ) + if (df.columns.contains(columnName)) { + df.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/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala index 9db9f8c1c4983..9d307db303aac 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala @@ -1,10 +1,12 @@ package org.apache.spark.graph.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 => +trait SharedCypherContext extends SharedSparkSession { + self: Suite => private var _cypherEngine: SparkCypherSession = _ @@ -12,6 +14,10 @@ trait SharedCypherContext extends SharedSparkSession { self: Suite => 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 diff --git a/pom.xml b/pom.xml index 5b6994ca6d722..e5c1e80d3826c 100644 --- a/pom.xml +++ b/pom.xml @@ -97,6 +97,8 @@ sql/catalyst sql/core sql/hive + graph/api + graph/cypher assembly examples repl From 59c1ac95ca3ed9aa39c6f38998527e865363b0e2 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 11:57:51 +0100 Subject: [PATCH 055/123] Move SparkCypherFunctions to impl root package Co-authored-by: Philip Stutz --- .../spark/graph/cypher/{udfs => }/SparkCypherFunctions.scala | 4 ++-- .../spark/graph/cypher/conversions/ExprConversions.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) rename graph/cypher/src/main/scala/org/apache/spark/graph/cypher/{udfs => }/SparkCypherFunctions.scala (95%) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala similarity index 95% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala rename to graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala index a3e8ebcb4b2c2..7f4cd25bed17c 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/SparkCypherFunctions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.cypher.udfs +package org.apache.spark.graph.cypher import org.apache.spark.graph.cypher.conversions.ExprConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue -import org.apache.spark.sql.catalyst.expressions.{ArrayContains, ArrayFilter, ArrayTransform, CaseWhen, EqualTo, Expression, GenericRowWithSchema, GetArrayItem, GetStructField, IsNotNull, LambdaFunction, NamedLambdaVariable, RLike, Slice, StringTranslate, XxHash64} +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, DataFrame, functions} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala index c26eb1dd79845..59e349d6670e0 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -3,7 +3,7 @@ package org.apache.spark.graph.cypher.conversions import org.apache.spark.sql.functions.{array_contains => _, translate => _, _} import org.apache.spark.graph.cypher.conversions.TemporalConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ -import org.apache.spark.graph.cypher.udfs.SparkCypherFunctions._ +import org.apache.spark.graph.cypher.SparkCypherFunctions._ import org.apache.spark.graph.cypher.udfs.TemporalUdfs import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame} From f80d489f41b843b0435f63b2c54ffb72de665acc Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 12:05:05 +0100 Subject: [PATCH 056/123] Update TCK blacklist Co-authored-by: Philip Stutz --- .../src/test/resources/tck/failing_blacklist | 51 +++---------------- 1 file changed, 7 insertions(+), 44 deletions(-) diff --git a/graph/cypher/src/test/resources/tck/failing_blacklist b/graph/cypher/src/test/resources/tck/failing_blacklist index 9db31748f2f36..bc3551537b015 100644 --- a/graph/cypher/src/test/resources/tck/failing_blacklist +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -30,21 +30,6 @@ 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 "TernaryLogicAcceptanceTest": Scenario "Using null in AND #1" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #2" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #3" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #4" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in AND #5" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #1" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #2" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #3" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #4" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in OR #5" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #1" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #2" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #3" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #4" -Feature "TernaryLogicAcceptanceTest": Scenario "Using null in XOR #5" 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" @@ -108,8 +93,6 @@ Feature "OptionalMatchAcceptance": Scenario "Variable length optional relationsh 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 "OptionalMatchAcceptance": Scenario "Handling optional matches between optionally matched entities" -Feature "OptionalMatchAcceptance": Scenario "Handling optional matches between nulls" Feature "MatchAcceptance2": Scenario "Aggregation with named paths" Feature "MatchAcceptance2": Scenario "Simple variable length pattern" Feature "MatchAcceptance2": Scenario "Variable length relationship without lower bound" @@ -120,10 +103,7 @@ Feature "MatchAcceptance2": Scenario "Optionally matching named paths with varia 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 "Simple OPTIONAL MATCH on empty graph" Feature "MatchAcceptance2": Scenario "Handling fixed-length variable length pattern" -Feature "MatchAcceptance2": Scenario "Optionally matching from null nodes should return null" -Feature "MatchAcceptance2": Scenario "OPTIONAL MATCH returns null" 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" @@ -173,7 +153,6 @@ Feature "MatchAcceptance": Scenario "Return relationships by collecting them as 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 "MatchAcceptance": Scenario "Accept skip 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" @@ -188,8 +167,6 @@ 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 "IN should work with list slices" -Feature "ExpressionAcceptance": Scenario "IN should work with literal list slices" 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" @@ -211,25 +188,26 @@ 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 "AggregationAcceptance": Scenario "`max()` should aggregate strings" +Feature "AggregationAcceptance": Scenario "`min()` should aggregate strings" Feature "Aggregation": Scenario "`min()` over mixed values" Feature "Aggregation": Scenario "`min()` over list values" Feature "Aggregation": Scenario "`max()` over mixed values" Feature "Aggregation": Scenario "`max()` over mixed numeric values" Feature "Aggregation": Scenario "`max()` over list values" +Feature "Aggregation": Scenario "`max()` over strings" +Feature "Aggregation": Scenario "`min()` over strings" 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 types of LHS and RHS don't match - singleton list" 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 "IN should return null if comparison with null is required for empty list" Feature "ListOperations": Scenario "IN should work with an empty list in the presence of other list elements: not matching" -Feature "ListOperations": Scenario "IN should work with list slices" 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 "IN should return false for the empty list if the LHS and RHS types differ" Feature "ListOperations": Scenario "IN should return false when LHS contains a nested list and type mismatch on RHS - singleton version" Feature "ListOperations": Scenario "Equality between list and literal should return false" Feature "ListOperations": Scenario "IN should return false when matching a list with a nested list with same elements" @@ -237,7 +215,6 @@ Feature "ListOperations": Scenario "IN should return false when LHS contains a n Feature "ListOperations": Scenario "IN should work with an empty list in the presence of other list elements: matching" Feature "ListOperations": Scenario "IN should return false if no match can be found, despite nulls" Feature "ListOperations": Scenario "Setting and returning the size of a list property" -Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected, starting from one extreme" 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" @@ -246,14 +223,11 @@ Feature "ListOperations": Scenario "IN should return true when types of LHS and 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 false when types of LHS and RHS don't match - list" -Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - directed, one way" 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 "IN should work with literal list slices" -Feature "MatchAcceptance": Scenario "Return relationships by collecting them as a list - undirected, starting from two extremes" 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" @@ -263,21 +237,10 @@ Feature "ListOperations": Scenario "IN should return true if correct list found 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 singleton range" -Feature "ListOperations": Scenario "List slice" Feature "ListOperations": Scenario "List slice with parameterised invalid range" -Feature "ListOperations": Scenario "List slice with implicit start" -Feature "ListOperations": Scenario "List slice with empty range" Feature "ListOperations": Scenario "List slice with invalid range" -Feature "ListOperations": Scenario "List slice with parameterised range" -Feature "ListOperations": Scenario "List slice with null range #1" -Feature "ListOperations": Scenario "List slice with null range #2" -Feature "ListOperations": Scenario "List slice with null range #3" -Feature "ListOperations": Scenario "List slice with null range #4" -Feature "ListOperations": Scenario "List slice with null range #5" -Feature "ListOperations": Scenario "List slice with implicit end" Feature "ListOperations": Scenario "List slice with exceeding range" Feature "NullAcceptance": Scenario "Property existence check on null node" -Feature "NullOperator": Scenario "Property null check on null node" -Feature "NullOperator": Scenario "Property not null check on null node" -Feature "ColumnNameAcceptance": Scenario "Keeping used expression 4" \ No newline at end of file +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" From db4498b89350a77984ede69734d8ef3d6f8d67df Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 12:07:18 +0100 Subject: [PATCH 057/123] Move nullSafeConversion to ExprConversions Co-authored-by: Philip Stutz --- .../graph/cypher/SparkCypherFunctions.scala | 26 ----------------- .../cypher/conversions/ExprConversions.scala | 29 ++++++++++++++++++- 2 files changed, 28 insertions(+), 27 deletions(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala index 7f4cd25bed17c..99dce1100742d 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala @@ -101,32 +101,6 @@ object SparkCypherFunctions { new Column(StringTranslate(src.expr, matchingString.expr, replaceString.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 null_safe_conversion(expr: Expr)(withConvertedChildren: Seq[Column] => Column) - (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { - if (expr.cypherType == CTNull) { - NULL_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) - } - } - } - def column_for(expr: Expr)(implicit header: RecordHeader, df: DataFrame): Column = { val columnName = header.getColumn(expr).getOrElse(throw IllegalArgumentException( expected = s"Expression in ${header.expressions.mkString("[", ", ", "]")}", diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala index 59e349d6670e0..bdaa47ae6bd78 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala @@ -5,6 +5,7 @@ import org.apache.spark.graph.cypher.conversions.TemporalConversions._ import org.apache.spark.graph.cypher.conversions.TypeConversions._ import org.apache.spark.graph.cypher.SparkCypherFunctions._ import org.apache.spark.graph.cypher.udfs.TemporalUdfs +import org.apache.spark.sql.catalyst.expressions.CaseWhen import org.apache.spark.sql.types._ import org.apache.spark.sql.{Column, DataFrame} import org.opencypher.okapi.api.types._ @@ -18,6 +19,32 @@ import org.opencypher.okapi.relational.impl.table.RecordHeader object ExprConversions { + /** + * 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, df: DataFrame, parameters: CypherMap): Column = { + if (expr.cypherType == CTNull) { + NULL_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) + } + } + } + implicit class RichExpression(expr: Expr) { /** @@ -33,7 +60,7 @@ object ExprConversions { // Evaluate based on already present data; no recursion case _: Var | _: HasLabel | _: HasType | _: StartNode | _: EndNode => column_for(expr) // Evaluate bottom-up - case _ => null_safe_conversion(expr)(convert) + case _ => nullSafeConversion(expr)(convert) } header.getColumn(expr) match { case None => outCol From cd9d712675a0f045b9faf866ecbcc31c0c5b6831 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 12:09:28 +0100 Subject: [PATCH 058/123] Remove references to CAPS Co-authored-by: Philip Stutz --- .../apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala index a6878dad94683..a139b27a3c17b 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -17,7 +17,7 @@ import scala.util.{Failure, Success, Try} class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { - object TckCapsTag extends Tag("TckSparkCypher") + private val tckSparkCypherTag = Tag("TckSparkCypher") private val graphFactory: CypherTestGraphFactory[SparkCypherSession] = ScanGraphFactory @@ -28,13 +28,13 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { private val scenarios = ScenariosFor(failingBlacklist, temporalBlacklist, wontFixBlacklistFile, failureReportingBlacklistFile) forAll(scenarios.whiteList) { scenario => - test(s"[${WhiteList.name}] $scenario", WhiteList, TckCapsTag, Tag(graphFactory.name)) { + 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, TckCapsTag) { + test(s"[${graphFactory.name}, ${BlackList.name}] $scenario", BlackList, tckSparkCypherTag) { val tckGraph = TCKGraph(graphFactory, internalCypherSession.graphs.empty)(internalCypherSession) Try(scenario(tckGraph).execute()) match { From fcb3aebd5d2550840d7e73d9bd762b10f88d63ef Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 29 Mar 2019 12:14:34 +0100 Subject: [PATCH 059/123] Close Source in TCK tests Co-authored-by: Philip Stutz --- .../graph/cypher/tck/SparkCypherTckSuite.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala index a139b27a3c17b..164db503d7a01 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala @@ -23,9 +23,9 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { private val failingBlacklist = getClass.getResource("/tck/failing_blacklist").getFile private val temporalBlacklist = getClass.getResource("/tck/temporal_blacklist").getFile - private val wontFixBlacklistFile = getClass.getResource("/tck/wont_fix_blacklist").getFile - private val failureReportingBlacklistFile = getClass.getResource("/tck/failure_reporting_blacklist").getFile - private val scenarios = ScenariosFor(failingBlacklist, temporalBlacklist, wontFixBlacklistFile, failureReportingBlacklistFile) + 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)) { @@ -46,9 +46,11 @@ class SparkCypherTckSuite extends SparkFunSuite with SharedCypherContext { } test("compute TCK coverage") { - val failingScenarios = Source.fromFile(failingBlacklist).getLines().size - val failingTemporalScenarios = Source.fromFile(temporalBlacklist).getLines().size - val failureReportingScenarios = Source.fromFile(failureReportingBlacklistFile).getLines().size + 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) From 54e7014e107004e8c61efda084229255533fac77 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 10:32:10 +0200 Subject: [PATCH 060/123] Extract schema methods into separate PropertyGraphType trait Co-authored-by: Philip Stutz --- .../spark/graph/api/PropertyGraph.scala | 14 ++---------- .../spark/graph/api/PropertyGraphType.scala | 22 +++++++++++++++++++ .../adapters/RelationalGraphAdapter.scala | 8 +++---- .../graph/cypher/adapters/SchemaAdapter.scala | 13 +++++++++++ 4 files changed, 40 insertions(+), 17 deletions(-) create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala 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 index 98abae0932b69..112dbe0a865aa 100644 --- 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 @@ -12,19 +12,9 @@ import org.apache.spark.sql.{DataFrame, SaveMode} trait PropertyGraph { /** - * Returns all labels occurring on any node in the graph. + * The schema (graph type) describes the structure of this graph. */ - def labels: Set[String] = labelSets.flatten - - /** - * Returns all distinct label sets occurring on nodes in the graph. - */ - def labelSets: Set[Set[String]] - - /** - * Returns all relationship types occurring on relationships in the graph. - */ - def relationshipTypes: Set[String] + def schema: PropertyGraphType /** * The session in which this graph is managed. diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala new file mode 100644 index 0000000000000..3a4ae25cffb81 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala @@ -0,0 +1,22 @@ +package org.apache.spark.graph.api + +/** + * Describes the structure of a [[PropertyGraph]]. + */ +trait PropertyGraphType { + /** + * Returns all labels occurring on any node in the graph. + */ + def labels: Set[String] = labelSets.flatten + + /** + * Returns all distinct label sets occurring on nodes in the graph. + */ + def labelSets: Set[Set[String]] + + /** + * Returns all relationship types occurring on relationships in the graph. + */ + def relationshipTypes: Set[String] + +} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala index 6c96395912f11..5cbfc85a51210 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala @@ -1,6 +1,6 @@ package org.apache.spark.graph.cypher.adapters -import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, PropertyGraphType, RelationshipFrame} import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.cypher.adapters.MappingAdapter._ import org.apache.spark.graph.cypher.{SparkCypherSession, SparkEntityTable} @@ -13,6 +13,8 @@ case class RelationalGraphAdapter( nodeFrames: Seq[NodeFrame], relationshipFrames: Seq[RelationshipFrame]) extends PropertyGraph { + override def schema: PropertyGraphType = SchemaAdapter(graph.schema) + private [graph] lazy val graph = { if (nodeFrames.isEmpty) { cypherSession.graphs.empty @@ -61,10 +63,6 @@ case class RelationalGraphAdapter( df.select(selectColumns: _*) } - override def labelSets: Set[Set[String]] = graph.schema.labelCombinations.combos - - override def relationshipTypes: Set[String] = graph.schema.relationshipTypes - override def nodeFrame(labelSet: Set[String]): NodeFrame = _nodeFrame(labelSet) override def relationshipFrame(relationshipType: String): RelationshipFrame = _relationshipFrame(relationshipType) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala new file mode 100644 index 0000000000000..1e59e3843c463 --- /dev/null +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala @@ -0,0 +1,13 @@ +package org.apache.spark.graph.cypher.adapters + +import org.apache.spark.graph.api.PropertyGraphType +import org.opencypher.okapi.api.schema.Schema + +case class SchemaAdapter(schema: Schema) extends PropertyGraphType { + + override def labelSets: Set[Set[String]] = schema.labelCombinations.combos + + override def relationshipTypes: Set[String] = schema.relationshipTypes + + +} From 34b595ff2663d02e4fb803d2e198b8c3291cff41 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 10:49:27 +0200 Subject: [PATCH 061/123] Update documentation Co-authored-by: Philip Stutz --- .../main/scala/org/apache/spark/graph/api/CypherSession.scala | 3 ++- .../scala/org/apache/spark/graph/api/GraphElementFrame.scala | 4 +++- 2 files changed, 5 insertions(+), 2 deletions(-) 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 index 25ac8e19c7f2d..0e033a37dc4fd 100644 --- 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 @@ -3,7 +3,8 @@ package org.apache.spark.graph.api import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} /** - * Allows for creating [[PropertyGraph]] instances and running Cypher-queries on them. + * Allows for creating and loading [[PropertyGraph]] instances and running Cypher-queries on them. + * Wraps a [[SparkSession]]. */ trait CypherSession { diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 6b12bdd9193cd..c63a2136cfd99 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -24,7 +24,9 @@ object GraphElementFrame { } /** - * Describes how to map an input [[DataFrame]] to graph elements (i.e. nodes or relationships). + * A [[PropertyGraph]] is created from GraphElementFrames. + * + * Wraps a [[DataFrame]] and describes how it maps to graph elements (i.e. nodes or relationships). */ trait GraphElementFrame { From 3af683a575f7a08ff08a65db4c039896741331f7 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 10:52:08 +0200 Subject: [PATCH 062/123] Rename spark-graph-cypher to spark-cypher Co-authored-by: Philip Stutz --- .../org/apache/spark/graph/cypher/SparkCypherSession.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala index 933b9407e06ad..f0f3bed0c3b32 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala @@ -20,7 +20,7 @@ object SparkCypherSession { /** * Default [[CypherSession]] implementation. * - * This class is the main entry point for working with the spark-graph-cypher module. + * 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.DataFrame]]s. */ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) extends RelationalCypherSession[DataFrameTable] with CypherSession { From 13e88890a50c6c7199e50633f01a296bfc4d535c Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 11:22:04 +0200 Subject: [PATCH 063/123] Rename org.apache.spark.graph.cypher to org.apache.spark.cypher Co-authored-by: Philip Stutz --- .../spark/{graph => }/cypher/SparkCypherEntity.scala | 4 ++-- .../{graph => }/cypher/SparkCypherFunctions.scala | 6 ++---- .../{graph => }/cypher/SparkCypherRecords.scala | 10 +++++----- .../spark/{graph => }/cypher/SparkCypherResult.scala | 4 ++-- .../{graph => }/cypher/SparkCypherSession.scala | 8 ++++---- .../spark/{graph => }/cypher/SparkEntityTable.scala | 4 ++-- .../cypher/SparkGraphDirectoryStructure.scala | 4 ++-- .../apache/spark/{graph => }/cypher/SparkTable.scala | 6 +++--- .../{graph => }/cypher/adapters/MappingAdapter.scala | 2 +- .../cypher/adapters/RelationalGraphAdapter.scala | 10 +++++----- .../{graph => }/cypher/adapters/SchemaAdapter.scala | 2 +- .../cypher/conversions/CypherValueEncoders.scala | 4 ++-- .../cypher/conversions/ExprConversions.scala | 12 ++++++------ .../cypher/conversions/RowConversion.scala | 6 +++--- .../cypher/conversions/StringEncodingUtilities.scala | 2 +- .../cypher/conversions/TemporalConversions.scala | 4 ++-- .../cypher/conversions/TypeConversions.scala | 4 ++-- .../spark/{graph => }/cypher/io/ReadWriteGraph.scala | 12 ++++++------ .../spark/{graph => }/cypher/udfs/TemporalUdfs.scala | 2 +- .../{graph => }/cypher/util/HadoopFSUtils.scala | 4 +--- .../GraphExamplesSuite.scala} | 11 +++++------ .../{graph => }/cypher/PropertyGraphReadWrite.scala | 2 +- .../{graph => }/cypher/SharedCypherContext.scala | 2 +- .../cypher/construction/ScanGraphFactory.scala | 10 +++++----- .../{graph => }/cypher/tck/SparkCypherTckSuite.scala | 6 +++--- 25 files changed, 68 insertions(+), 73 deletions(-) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkCypherEntity.scala (94%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkCypherFunctions.scala (95%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkCypherRecords.scala (90%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkCypherResult.scala (76%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkCypherSession.scala (95%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkEntityTable.scala (84%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkGraphDirectoryStructure.scala (94%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/SparkTable.scala (98%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/adapters/MappingAdapter.scala (95%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/adapters/RelationalGraphAdapter.scala (91%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/adapters/SchemaAdapter.scala (87%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/conversions/CypherValueEncoders.scala (88%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/conversions/ExprConversions.scala (98%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/conversions/RowConversion.scala (95%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/conversions/StringEncodingUtilities.scala (98%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/conversions/TemporalConversions.scala (97%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/conversions/TypeConversions.scala (98%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/io/ReadWriteGraph.scala (91%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/udfs/TemporalUdfs.scala (99%) rename graph/cypher/src/main/scala/org/apache/spark/{graph => }/cypher/util/HadoopFSUtils.scala (85%) rename graph/cypher/src/test/scala/org/apache/spark/{graph/cypher/BasicMatchSuite.scala => cypher/GraphExamplesSuite.scala} (90%) rename graph/cypher/src/test/scala/org/apache/spark/{graph => }/cypher/PropertyGraphReadWrite.scala (97%) rename graph/cypher/src/test/scala/org/apache/spark/{graph => }/cypher/SharedCypherContext.scala (95%) rename graph/cypher/src/test/scala/org/apache/spark/{graph => }/cypher/construction/ScanGraphFactory.scala (94%) rename graph/cypher/src/test/scala/org/apache/spark/{graph => }/cypher/tck/SparkCypherTckSuite.scala (95%) diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala similarity index 94% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala index 36c054fe4ffd5..4eee879acb0b0 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherEntity.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher -import org.apache.spark.graph.cypher.SparkCypherEntity._ +import org.apache.spark.cypher.SparkCypherEntity._ import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherNode, CypherRelationship} object SparkCypherEntity { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala similarity index 95% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala index 99dce1100742d..343555f17e73a 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherFunctions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala @@ -1,13 +1,11 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher -import org.apache.spark.graph.cypher.conversions.ExprConversions._ -import org.apache.spark.graph.cypher.conversions.TypeConversions._ +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, DataFrame, functions} -import org.opencypher.okapi.api.types.CTNull import org.opencypher.okapi.api.types.CypherType._ import org.opencypher.okapi.api.value.CypherValue.{CypherList, CypherMap, CypherValue} import org.opencypher.okapi.impl.exception.IllegalArgumentException diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala similarity index 90% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala index 5cf1eac819ea4..8699618a5083f 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherRecords.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher import java.util.Collections -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.graph.cypher.conversions.RowConversion -import org.apache.spark.graph.cypher.conversions.TypeConversions._ -import org.apache.spark.graph.cypher.conversions.CypherValueEncoders._ +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} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala similarity index 76% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala index 2ca1442ee5156..9814c7be32f29 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher +import org.apache.spark.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.api.CypherResult -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable import org.apache.spark.sql.DataFrame import org.opencypher.okapi.relational.api.table.RelationalCypherRecords diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala similarity index 95% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala index f0f3bed0c3b32..9abd35b009dee 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher +import org.apache.spark.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.adapters.RelationalGraphAdapter +import org.apache.spark.cypher.io.ReadWriteGraph._ import org.apache.spark.graph.api._ -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.graph.cypher.adapters.RelationalGraphAdapter -import org.apache.spark.graph.cypher.io.ReadWriteGraph._ import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, functions} import org.opencypher.okapi.api.value.CypherValue.CypherMap import org.opencypher.okapi.impl.exception.{IllegalArgumentException, UnsupportedOperationException} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala similarity index 84% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala index 4dcb1f3956853..894694135c9fd 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkEntityTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable +import org.apache.spark.cypher.SparkTable.DataFrameTable import org.opencypher.okapi.api.io.conversion.EntityMapping import org.opencypher.okapi.relational.api.io.EntityTable diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala similarity index 94% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala index 814a452763e1d..2d92f6a6202ba 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkGraphDirectoryStructure.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala @@ -1,10 +1,10 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher import org.apache.hadoop.fs.Path object SparkGraphDirectoryStructure { - import org.apache.spark.graph.cypher.conversions.StringEncodingUtilities._ + import org.apache.spark.cypher.conversions.StringEncodingUtilities._ private implicit class StringPath(val path: String) extends AnyVal { def /(segment: String): String = s"$path$pathSeparator$segment" diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala similarity index 98% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala index 1090bd24da94b..bfe86ede9b5b3 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher -import org.apache.spark.graph.cypher.conversions.ExprConversions._ -import org.apache.spark.graph.cypher.conversions.TypeConversions._ +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.{Column, DataFrame, RelationalGroupedDataset, functions} import org.apache.spark.storage.StorageLevel diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/MappingAdapter.scala similarity index 95% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/MappingAdapter.scala index b9dfc60f8ccd5..927a0d1cb8100 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/MappingAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/MappingAdapter.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.cypher.adapters +package org.apache.spark.cypher.adapters import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMappingBuilder, RelationshipMappingBuilder} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/RelationalGraphAdapter.scala similarity index 91% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/RelationalGraphAdapter.scala index 5cbfc85a51210..68b297a3281a9 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/RelationalGraphAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/RelationalGraphAdapter.scala @@ -1,9 +1,9 @@ -package org.apache.spark.graph.cypher.adapters +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.{SparkCypherSession, SparkEntityTable} import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, PropertyGraphType, RelationshipFrame} -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.graph.cypher.adapters.MappingAdapter._ -import org.apache.spark.graph.cypher.{SparkCypherSession, SparkEntityTable} import org.apache.spark.sql.DataFrame import org.opencypher.okapi.api.types.{CTNode, CTRelationship} import org.opencypher.okapi.ir.api.expr.Var @@ -15,7 +15,7 @@ case class RelationalGraphAdapter( override def schema: PropertyGraphType = SchemaAdapter(graph.schema) - private [graph] lazy val graph = { + private[cypher] lazy val graph = { if (nodeFrames.isEmpty) { cypherSession.graphs.empty } else { diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/SchemaAdapter.scala similarity index 87% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/SchemaAdapter.scala index 1e59e3843c463..42aa7019b4406 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/adapters/SchemaAdapter.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/adapters/SchemaAdapter.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.cypher.adapters +package org.apache.spark.cypher.adapters import org.apache.spark.graph.api.PropertyGraphType import org.opencypher.okapi.api.schema.Schema diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/CypherValueEncoders.scala similarity index 88% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/CypherValueEncoders.scala index 67659b3db16a5..4f123e98f26e0 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/CypherValueEncoders.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/CypherValueEncoders.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.cypher.conversions +package org.apache.spark.cypher.conversions -import org.apache.spark.graph.cypher.{SparkCypherNode, SparkCypherRelationship} +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 diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/ExprConversions.scala similarity index 98% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/ExprConversions.scala index bdaa47ae6bd78..2ccd37009e0e0 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/ExprConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/ExprConversions.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.cypher.conversions +package org.apache.spark.cypher.conversions -import org.apache.spark.sql.functions.{array_contains => _, translate => _, _} -import org.apache.spark.graph.cypher.conversions.TemporalConversions._ -import org.apache.spark.graph.cypher.conversions.TypeConversions._ -import org.apache.spark.graph.cypher.SparkCypherFunctions._ -import org.apache.spark.graph.cypher.udfs.TemporalUdfs +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, DataFrame} import org.opencypher.okapi.api.types._ diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala similarity index 95% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala index c1649021d7e44..5e5854ecee594 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/RowConversion.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala @@ -1,7 +1,7 @@ -package org.apache.spark.graph.cypher.conversions +package org.apache.spark.cypher.conversions -import org.apache.spark.graph.cypher.conversions.TemporalConversions._ -import org.apache.spark.graph.cypher.{SparkCypherNode, SparkCypherRelationship} +import org.apache.spark.cypher.conversions.TemporalConversions._ +import org.apache.spark.cypher.{SparkCypherNode, SparkCypherRelationship} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.unsafe.types.CalendarInterval diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/StringEncodingUtilities.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/StringEncodingUtilities.scala similarity index 98% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/StringEncodingUtilities.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/StringEncodingUtilities.scala index ced0464448fba..ed0e8726072f9 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/StringEncodingUtilities.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/StringEncodingUtilities.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.cypher.conversions +package org.apache.spark.cypher.conversions import scala.annotation.tailrec diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TemporalConversions.scala similarity index 97% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TemporalConversions.scala index 8669374061be6..19b07ecdd6e9e 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TemporalConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TemporalConversions.scala @@ -1,8 +1,8 @@ -package org.apache.spark.graph.cypher.conversions +package org.apache.spark.cypher.conversions import java.time.temporal.ChronoUnit -import org.apache.spark.graph.cypher.udfs.TemporalUdfs +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} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TypeConversions.scala similarity index 98% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TypeConversions.scala index 372a706b09742..43f1e07ab9b08 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/conversions/TypeConversions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/TypeConversions.scala @@ -1,6 +1,6 @@ -package org.apache.spark.graph.cypher.conversions +package org.apache.spark.cypher.conversions -import org.apache.spark.graph.cypher.conversions.TemporalConversions._ +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._ diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala similarity index 91% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala index 8b680584711b3..03f80c03eab35 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/io/ReadWriteGraph.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala @@ -1,13 +1,13 @@ -package org.apache.spark.graph.cypher.io +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.{NodeFrame, RelationshipFrame} -import org.apache.spark.graph.cypher.SparkGraphDirectoryStructure -import org.apache.spark.graph.cypher.SparkGraphDirectoryStructure._ -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.graph.cypher.conversions.StringEncodingUtilities._ import org.apache.spark.sql.{DataFrame, SparkSession} import org.opencypher.okapi.api.graph.{SourceEndNodeKey, SourceIdKey, SourceStartNodeKey} import org.opencypher.okapi.api.types.{CTNode, CTRelationship} @@ -22,7 +22,7 @@ object ReadWriteGraph { val (labelCombos, relTypes): (Seq[Set[String]], Seq[String]) = { val fs = FileSystem.get(new URI(path), sparkSession.sparkContext.hadoopConfiguration) try { - import org.apache.spark.graph.cypher.util.HadoopFSUtils._ + 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 diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/udfs/TemporalUdfs.scala similarity index 99% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/udfs/TemporalUdfs.scala index 9aba46794276a..2b066f0c9ebd3 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/udfs/TemporalUdfs.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/udfs/TemporalUdfs.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.cypher.udfs +package org.apache.spark.cypher.udfs import java.sql.{Date, Timestamp} import java.time.temporal.{ChronoField, IsoFields, TemporalField} diff --git a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/util/HadoopFSUtils.scala similarity index 85% rename from graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala rename to graph/cypher/src/main/scala/org/apache/spark/cypher/util/HadoopFSUtils.scala index 35a72190c3340..6fc141394a609 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/graph/cypher/util/HadoopFSUtils.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/util/HadoopFSUtils.scala @@ -1,9 +1,7 @@ -package org.apache.spark.graph.cypher.util +package org.apache.spark.cypher.util import org.apache.hadoop.fs.{FileSystem, Path} -import scala.util.Try - object HadoopFSUtils { implicit class RichHadoopFileSystem(fileSystem: FileSystem) { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala similarity index 90% rename from graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala rename to graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala index 2e017c2c0981c..f0b90c3572a48 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/BasicMatchSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -1,18 +1,17 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} import org.apache.spark.sql.DataFrame -class BasicMatchSuite extends SparkFunSuite with SharedCypherContext { +class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("match single node pattern using spark-graph-api") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame)) - - graph.cypher("MATCH (n) RETURN n").df.show() + val result: CypherResult = graph.cypher("MATCH (n) RETURN n") + result.df.show() } test("match simple pattern using spark-graph-api") { diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala similarity index 97% rename from graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala rename to graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala index 9bc4248389f5b..a9ed9f62921b6 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher import java.nio.file.Paths diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala similarity index 95% rename from graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala rename to graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala index 9d307db303aac..27013e52ac1d5 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala @@ -1,4 +1,4 @@ -package org.apache.spark.graph.cypher +package org.apache.spark.cypher import org.apache.spark.SparkConf import org.apache.spark.graph.api.CypherSession diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala similarity index 94% rename from graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala rename to graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala index 693b83f9b2e0b..4360dc6b667f8 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/construction/ScanGraphFactory.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala @@ -1,11 +1,11 @@ -package org.apache.spark.graph.cypher.construction +package org.apache.spark.cypher.construction import java.time.{LocalDate, LocalDateTime} -import org.apache.spark.graph.cypher.SparkTable.DataFrameTable -import org.apache.spark.graph.cypher.conversions.TemporalConversions._ -import org.apache.spark.graph.cypher.conversions.TypeConversions._ -import org.apache.spark.graph.cypher.{SparkCypherSession, SparkEntityTable} +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.{DataFrame, Row} import org.opencypher.okapi.api.graph.Pattern diff --git a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/tck/SparkCypherTckSuite.scala similarity index 95% rename from graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala rename to graph/cypher/src/test/scala/org/apache/spark/cypher/tck/SparkCypherTckSuite.scala index 164db503d7a01..38957e27049ac 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/graph/cypher/tck/SparkCypherTckSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/tck/SparkCypherTckSuite.scala @@ -1,10 +1,10 @@ -package org.apache.spark.graph.cypher.tck +package org.apache.spark.cypher.tck import java.io.File import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.cypher.construction.ScanGraphFactory -import org.apache.spark.graph.cypher.{SharedCypherContext, SparkCypherSession} +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 From cc9c61710968b3fe012466ec2ce7759367b3f05f Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 12:02:46 +0200 Subject: [PATCH 064/123] Document and check node/rel frame uniqueness on graph creation Co-authored-by: Philip Stutz --- .../apache/spark/graph/api/CypherSession.scala | 3 +++ .../spark/graph/api/GraphElementFrame.scala | 17 +++++++++++------ .../spark/cypher/SparkCypherSession.scala | 4 ++++ .../spark/cypher/adapters/MappingAdapter.scala | 2 +- .../adapters/RelationalGraphAdapter.scala | 2 +- 5 files changed, 20 insertions(+), 8 deletions(-) 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 index 0e033a37dc4fd..fbe3217af2126 100644 --- 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 @@ -31,6 +31,9 @@ trait CypherSession { * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. * At least one [[NodeFrame]] has to be provided. * + * For each label set and relationship type there can be ar most on [[NodeFrame]] and + * [[RelationshipFrame]], respectively. + * * @param nodes [[NodeFrame]]s that define the nodes in the graph * @param relationships [[RelationshipFrame]]s that define the relationships in the graph */ diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index c63a2136cfd99..612710a9743b8 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -97,15 +97,18 @@ object NodeFrame { /** * Describes how to map an input [[DataFrame]] to nodes. * + * Each row in the [[DataFrame]] represents a node which has exactly the labels defined by the + * given label set. + * * @param initialDf [[DataFrame]] containing a single node in each row * @param idColumn column that contains the node identifier - * @param labels labels that are assigned to all nodes + * @param labelSet labels that are assigned to all nodes * @param properties mapping from property keys to corresponding columns */ case class NodeFrame( initialDf: DataFrame, idColumn: String, - labels: Set[String], + labelSet: Set[String], properties: Map[String, String] ) extends GraphElementFrame { @@ -120,22 +123,22 @@ object RelationshipFrame { * * All columns apart from the given identifier columns are mapped to relationship properties. * - * @param initialDf [[DataFrame]] containing a single relationship in each row + * @param df [[DataFrame]] 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 */ def apply( - initialDf: DataFrame, + df: DataFrame, idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String ): RelationshipFrame = { - val properties = (initialDf.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) + val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) .map(columnName => columnName -> columnName).toMap - RelationshipFrame(initialDf, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) } } @@ -143,6 +146,8 @@ object RelationshipFrame { /** * Describes how to map an input [[DataFrame]] to relationships. * + * Each row in the [[DataFrame]] represents a relationship with the given relationship type. + * * @param initialDf [[DataFrame]] 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 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 index 9abd35b009dee..e22e3b0b4ad69 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -43,6 +43,10 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) } override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph = { + require(nodes.groupBy(_.labelSet).forall(_._2.size == 1), + "There can be at most one NodeFrame per label set") + require(relationships.groupBy(_.relationshipType).forall(_._2.size == 1), + "There can be at most one RelationshipFrame per relationship type") RelationalGraphAdapter(this, nodes, relationships) } 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 index 927a0d1cb8100..06f6cc8754e75 100644 --- 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 @@ -8,7 +8,7 @@ object MappingAdapter { implicit class RichNodeDataFrame(val nodeDf: NodeFrame) extends AnyVal { def toNodeMapping: EntityMapping = NodeMappingBuilder .on(nodeDf.idColumn) - .withImpliedLabels(nodeDf.labels.toSeq: _*) + .withImpliedLabels(nodeDf.labelSet.toSeq: _*) .withPropertyKeyMappings(nodeDf.properties.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 index 68b297a3281a9..06d62eae8c1e2 100644 --- 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 @@ -25,7 +25,7 @@ case class RelationalGraphAdapter( } } - private lazy val _nodeFrame: Map[Set[String], NodeFrame] = nodeFrames.map(nf => nf.labels -> nf).toMap + private lazy val _nodeFrame: Map[Set[String], NodeFrame] = nodeFrames.map(nf => nf.labelSet -> nf).toMap private lazy val _relationshipFrame: Map[String, RelationshipFrame] = relationshipFrames.map(rf => rf.relationshipType -> rf).toMap From 0ca9c00571376e1c2366bd836143496f8ca780b9 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 12:13:52 +0200 Subject: [PATCH 065/123] Add default implementation for graph creation from two DFs to API Co-authored-by: Philip Stutz --- .../spark/graph/api/CypherSession.scala | 34 +++++++++++++++++-- .../spark/cypher/SparkCypherSession.scala | 32 ----------------- 2 files changed, 32 insertions(+), 34 deletions(-) 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 index fbe3217af2126..0d87d15d84003 100644 --- 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 @@ -1,6 +1,6 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, functions} /** * Allows for creating and loading [[PropertyGraph]] instances and running Cypher-queries on them. @@ -58,7 +58,37 @@ trait CypherSession { * @param nodes node [[DataFrame]] * @param relationships relationship [[DataFrame]] */ - def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph + def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { + val idColumn = "$ID" + val sourceIdColumn = "$SOURCE_ID" + val targetIdColumn = "$TARGET_ID" + + val labelColumns = nodes.columns.filter(_.startsWith(":")).toSet + val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns).map(col => col -> col).toMap + + val trueLit = functions.lit(true) + val falseLit = functions.lit(false) + + // TODO: add empty set + val nodeFrames = labelColumns.subsets().map { labelSet => + val predicate = labelColumns.map { + case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) === trueLit + case labelColumn => nodes.col(labelColumn) === falseLit + }.reduce(_ && _) + + NodeFrame(nodes.filter(predicate), idColumn, labelSet.map(_.substring(1)), nodeProperties) + } + + val relTypeColumns = relationships.columns.filter(_.startsWith(":")).toSet + val relProperties = (relationships.columns.toSet - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns).map(col => col -> col).toMap + val relFrames = relTypeColumns.map { relTypeColumn => + val predicate = relationships.col(relTypeColumn) === trueLit + + RelationshipFrame(relationships.filter(predicate), idColumn, sourceIdColumn, targetIdColumn, relTypeColumn.substring(1), relProperties) + } + + createGraph(nodeFrames.toSeq, relFrames.toSeq) + } /** * Loads a [[PropertyGraph]] from the given location. 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 index e22e3b0b4ad69..875a4242df524 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -50,38 +50,6 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) RelationalGraphAdapter(this, nodes, relationships) } - override def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { - val idColumn = "$ID" - val sourceIdColumn = "$SOURCE_ID" - val targetIdColumn = "$TARGET_ID" - - val labelColumns = nodes.columns.filter(_.startsWith(":")).toSet - val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns).map(col => col -> col).toMap - - val trueLit = functions.lit(true) - val falseLit = functions.lit(false) - - // TODO: add empty set - val nodeFrames = labelColumns.subsets().map { labelSet => - val predicate = labelColumns.map { - case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) === trueLit - case labelColumn => nodes.col(labelColumn) === falseLit - }.reduce(_ && _) - - NodeFrame(nodes.filter(predicate), idColumn, labelSet.map(_.substring(1)), nodeProperties) - } - - val relTypeColumns = relationships.columns.filter(_.startsWith(":")).toSet - val relProperties = (relationships.columns.toSet - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns).map(col => col -> col).toMap - val relFrames = relTypeColumns.map { relTypeColumn => - val predicate = relationships.col(relTypeColumn) === trueLit - - RelationshipFrame(relationships.filter(predicate), idColumn, sourceIdColumn, targetIdColumn, relTypeColumn.substring(1), relProperties) - } - - createGraph(nodeFrames.toSeq, relFrames.toSeq) - } - def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) override def cypher(graph: PropertyGraph, query: String, parameters: Map[String, Any]): CypherResult = { From 73156e88dd8c7ce5e6e91c42448f5d34909fdac8 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 3 Apr 2019 12:31:05 +0200 Subject: [PATCH 066/123] Add TODOs Co-authored-by: Philip Stutz --- .../apache/spark/cypher/adapters/RelationalGraphAdapter.scala | 2 ++ 1 file changed, 2 insertions(+) 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 index 06d62eae8c1e2..5497d3b6b2e95 100644 --- 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 @@ -30,6 +30,7 @@ case class RelationalGraphAdapter( private lazy val _relationshipFrame: Map[String, RelationshipFrame] = relationshipFrames.map(rf => rf.relationshipType -> rf).toMap override def nodes: DataFrame = { + // TODO: move to API as default implementation val nodeVar = Var("n")(CTNode) val nodes = graph.nodes(nodeVar.name) @@ -46,6 +47,7 @@ case class RelationalGraphAdapter( } override def relationships: DataFrame = { + // TODO: move to API as default implementation val relVar = Var("r")(CTRelationship) val rels = graph.relationships(relVar.name) From 08d81eaeca3990a1602fa5f5f09d261accd03d0f Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 4 Apr 2019 10:39:27 +0200 Subject: [PATCH 067/123] Add empty label set to wide table extraction MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling Co-authored-by: Philip Stutz --- .../main/scala/org/apache/spark/graph/api/CypherSession.scala | 4 ++-- .../scala/org/apache/spark/cypher/SparkCypherSession.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 index 0d87d15d84003..6a31bc14e4386 100644 --- 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 @@ -69,8 +69,8 @@ trait CypherSession { val trueLit = functions.lit(true) val falseLit = functions.lit(false) - // TODO: add empty set - val nodeFrames = labelColumns.subsets().map { labelSet => + val labelSets = labelColumns.subsets().toSet + Set.empty + val nodeFrames = labelSets.map { labelSet => val predicate = labelColumns.map { case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) === trueLit case labelColumn => nodes.col(labelColumn) === falseLit 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 index 875a4242df524..4dcbe1cdf5102 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -4,7 +4,7 @@ import org.apache.spark.cypher.SparkTable.DataFrameTable import org.apache.spark.cypher.adapters.RelationalGraphAdapter import org.apache.spark.cypher.io.ReadWriteGraph._ import org.apache.spark.graph.api._ -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, functions} +import org.apache.spark.sql.{SaveMode, 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} From ba75f5554ccf4c70f693021bb20aae94d8f2aeae Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 4 Apr 2019 12:10:18 +0200 Subject: [PATCH 068/123] Add examples aligned with design document MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../spark/graph/api/PropertyGraph.scala | 9 ++- .../spark/cypher/GraphExamplesSuite.scala | 57 ++++++++++++++++--- 2 files changed, 58 insertions(+), 8 deletions(-) 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 index 112dbe0a865aa..2f9c5b17988d1 100644 --- 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 @@ -21,13 +21,20 @@ trait PropertyGraph { */ 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 + */ + def cypher(query: String): CypherResult = cypher(query, Map.empty) + /** * 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 */ - def cypher(query: String, parameters: Map[String, Any] = Map.empty): CypherResult = cypherSession.cypher(this, query, parameters) + def cypher(query: String, parameters: Map[String, Any]): CypherResult = cypherSession.cypher(this, query, parameters) /** * Returns the [[NodeFrame]] for a given node label set. 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 index f0b90c3572a48..e7aafe7feb176 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -6,7 +6,7 @@ import org.apache.spark.sql.DataFrame class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { - test("match single node pattern using spark-graph-api") { + test("create graph with nodes") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame)) @@ -14,17 +14,60 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { result.df.show() } - test("match simple pattern using spark-graph-api") { + test("create graph with nodes and relationships") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") - val relationshipData: DataFrame = spark.createDataFrame(Seq(Tuple3(0, 0, 1))).toDF("id", "source", "target") + val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) - graph.nodes.show() - graph.relationships.show() + val result: CypherResult = graph.cypher( + """ + |MATCH (a:Person)-[r:KNOWS]->(:Person) + |RETURN a, r""".stripMargin) + result.df.show() + } + + test("create graph with multiple node and relationship types") { + val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") + val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") + + val studentNF: NodeFrame = NodeFrame(df = studentDF, idColumn = "id", labels = Set("Person", "Student")) + val teacherNF: NodeFrame = NodeFrame(df = teacherDF, idColumn = "id", labels = Set("Person", "Teacher")) - graph.cypher("MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2").df.show() + val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") + val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") + + val knowsRF: RelationshipFrame = RelationshipFrame(df = knowsDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val teachesRF: RelationshipFrame = RelationshipFrame(df = teachesDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "TEACHES") + + val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") + result.df.show() + } + + test("create graph with multiple node and relationship types from wide tables") { + val nodeDF: DataFrame = 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: DataFrame = 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.df.show() + } + + test("save and load Property Graph") { + val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) + graph1.nodes.show() + graph1.save("/tmp/my-storage") + val graph2: PropertyGraph = cypherSession.load("/tmp/my-storage") + graph2.nodes.show() } test("round trip example using column name conventions") { From 5ee6d043edb1d1e5f7f25db8c486fe162132f049 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 4 Apr 2019 15:33:53 +0200 Subject: [PATCH 069/123] Add example for custom column mapping --- .../spark/graph/api/GraphElementFrame.scala | 22 ++++++------ .../spark/cypher/GraphExamplesSuite.scala | 34 ++++++++++++++----- .../spark/cypher/PropertyGraphReadWrite.scala | 2 +- 3 files changed, 38 insertions(+), 20 deletions(-) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 612710a9743b8..0c562fe9629d5 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -78,18 +78,18 @@ object NodeFrame { * * All columns apart from the given `idColumn` are mapped to node properties. * - * @param df [[DataFrame]] containing a single node in each row - * @param idColumn column that contains the node identifier - * @param labels labels that are assigned to all nodes + * @param initialDf [[DataFrame]] containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labelSet labels that are assigned to all nodes */ def apply( - df: DataFrame, + initialDf: DataFrame, idColumn: String, - labels: Set[String] = Set.empty + labelSet: Set[String] = Set.empty ): NodeFrame = { - val properties = (df.columns.toSet - idColumn) + val properties = (initialDf.columns.toSet - idColumn) .map(columnName => columnName -> columnName).toMap - NodeFrame(df, idColumn, labels, properties) + NodeFrame(initialDf, idColumn, labelSet, properties) } } @@ -123,22 +123,22 @@ object RelationshipFrame { * * All columns apart from the given identifier columns are mapped to relationship properties. * - * @param df [[DataFrame]] containing a single relationship in each row + * @param initialDf [[DataFrame]] 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 */ def apply( - df: DataFrame, + initialDf: DataFrame, idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String ): RelationshipFrame = { - val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) + val properties = (initialDf.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) .map(columnName => columnName -> columnName).toMap - RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + RelationshipFrame(initialDf, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) } } 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 index e7aafe7feb176..0ad0448b84a34 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -8,8 +8,8 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create graph with nodes") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") - val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame)) + val nodeFrame: NodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) + val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame)) val result: CypherResult = graph.cypher("MATCH (n) RETURN n") result.df.show() } @@ -17,9 +17,9 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create graph with nodes and relationships") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") - val nodeDataFrame: NodeFrame = NodeFrame(df = nodeData, idColumn = "id", labels = Set("Person")) + val nodeFrame: NodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) val result: CypherResult = graph.cypher( """ |MATCH (a:Person)-[r:KNOWS]->(:Person) @@ -31,14 +31,32 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") - val studentNF: NodeFrame = NodeFrame(df = studentDF, idColumn = "id", labels = Set("Person", "Student")) - val teacherNF: NodeFrame = NodeFrame(df = teacherDF, idColumn = "id", labels = Set("Person", "Teacher")) + val studentNF: NodeFrame = NodeFrame(initialDf = studentDF, idColumn = "id", labelSet = Set("Person", "Student")) + val teacherNF: NodeFrame = NodeFrame(initialDf = teacherDF, idColumn = "id", labelSet = Set("Person", "Teacher")) val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") - val knowsRF: RelationshipFrame = RelationshipFrame(df = knowsDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val teachesRF: RelationshipFrame = RelationshipFrame(df = teachesDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "TEACHES") + val knowsRF: RelationshipFrame = RelationshipFrame(initialDf = knowsDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val teachesRF: RelationshipFrame = RelationshipFrame(initialDf = teachesDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "TEACHES") + + val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") + result.df.show() + } + + test("create graph with multiple node and relationship types and explicit property-to-column mappings") { + val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "col_name", "col_age") + val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "col_name", "col_subject") + + val studentNF: NodeFrame = NodeFrame(initialDf = studentDF, idColumn = "id", labelSet = Set("Person", "Student"), properties = Map("name" -> "col_name", "age" -> "col_age")) + val teacherNF: NodeFrame = NodeFrame(initialDf = teacherDF, idColumn = "id", labelSet = Set("Person", "Teacher"), properties = Map("name" -> "col_name", "subject" -> "col_subject")) + + val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "col_since") + val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") + + val knowsRF: RelationshipFrame = RelationshipFrame(initialDf = knowsDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS", properties = Map("since" -> "col_since")) + val teachesRF: RelationshipFrame = RelationshipFrame(initialDf = teachesDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "TEACHES") val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") 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 index a9ed9f62921b6..d3020110a15d9 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -35,7 +35,7 @@ class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with )).toDF("id", "source", "target") private lazy val nodeDataFrame: NodeFrame = NodeFrame( - df = nodeData, idColumn = "id", labels = Set("Person") + initialDf = nodeData, idColumn = "id", labelSet = Set("Person") ) private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame( From b2e9e93872db82e92e508f13087e4269ffd81604 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 4 Apr 2019 15:49:30 +0200 Subject: [PATCH 070/123] Clean up examples --- .../apache/spark/cypher/GraphExamplesSuite.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) 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 index 0ad0448b84a34..6a68213511601 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -2,11 +2,11 @@ package org.apache.spark.cypher import org.apache.spark.SparkFunSuite import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, SaveMode} class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { - test("create graph with nodes") { + test("create PropertyGraph from single NodeFrame") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeFrame: NodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame)) @@ -14,7 +14,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { result.df.show() } - test("create graph with nodes and relationships") { + test("create PropertyGraph from Node- and RelationshipFrames") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") val nodeFrame: NodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) @@ -27,7 +27,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { result.df.show() } - test("create graph with multiple node and relationship types") { + test("create PropertyGraph with multiple node and relationship types") { val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") @@ -45,7 +45,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { result.df.show() } - test("create graph with multiple node and relationship types and explicit property-to-column mappings") { + test("create PropertyGraph with multiple node and relationship types and explicit property-to-column mappings") { val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "col_name", "col_age") val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "col_name", "col_subject") @@ -63,7 +63,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { result.df.show() } - test("create graph with multiple node and relationship types from wide tables") { + test("create PropertyGraph with multiple node and relationship types stored in wide tables") { val nodeDF: DataFrame = spark.createDataFrame(Seq( (0L, true, true, false, Some("Alice"), Some(42), None), (1L, true, true, false, Some("Bob"), Some(23), None), @@ -80,10 +80,10 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { result.df.show() } - test("save and load Property Graph") { + test("save and load PropertyGraph") { val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) graph1.nodes.show() - graph1.save("/tmp/my-storage") + graph1.save("/tmp/my-storage", SaveMode.Overwrite) val graph2: PropertyGraph = cypherSession.load("/tmp/my-storage") graph2.nodes.show() } From 557715e203b5a3b31cb0780f0cea12795faeedab Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Sun, 7 Apr 2019 11:27:26 +0200 Subject: [PATCH 071/123] Add license headers --- dev/.rat-excludes | 1 + .../apache/spark/graph/api/CypherResult.scala | 18 +++++++++++++++++ .../spark/graph/api/CypherSession.scala | 18 +++++++++++++++++ .../spark/graph/api/GraphElementFrame.scala | 18 +++++++++++++++++ .../spark/graph/api/PropertyGraph.scala | 18 +++++++++++++++++ .../spark/graph/api/PropertyGraphType.scala | 18 +++++++++++++++++ .../spark/cypher/SparkCypherEntity.scala | 18 +++++++++++++++++ .../spark/cypher/SparkCypherFunctions.scala | 18 +++++++++++++++++ .../spark/cypher/SparkCypherRecords.scala | 18 +++++++++++++++++ .../spark/cypher/SparkCypherResult.scala | 18 +++++++++++++++++ .../spark/cypher/SparkCypherSession.scala | 18 +++++++++++++++++ .../spark/cypher/SparkEntityTable.scala | 18 +++++++++++++++++ .../cypher/SparkGraphDirectoryStructure.scala | 18 +++++++++++++++++ .../org/apache/spark/cypher/SparkTable.scala | 18 +++++++++++++++++ .../cypher/adapters/MappingAdapter.scala | 18 +++++++++++++++++ .../adapters/RelationalGraphAdapter.scala | 18 +++++++++++++++++ .../spark/cypher/adapters/SchemaAdapter.scala | 18 +++++++++++++++++ .../conversions/CypherValueEncoders.scala | 18 +++++++++++++++++ .../cypher/conversions/ExprConversions.scala | 18 +++++++++++++++++ .../cypher/conversions/RowConversion.scala | 18 +++++++++++++++++ .../conversions/StringEncodingUtilities.scala | 18 +++++++++++++++++ .../conversions/TemporalConversions.scala | 18 +++++++++++++++++ .../cypher/conversions/TypeConversions.scala | 18 +++++++++++++++++ .../spark/cypher/io/ReadWriteGraph.scala | 18 +++++++++++++++++ .../spark/cypher/udfs/TemporalUdfs.scala | 18 +++++++++++++++++ .../spark/cypher/util/HadoopFSUtils.scala | 18 +++++++++++++++++ .../spark/cypher/GraphExamplesSuite.scala | 20 ++++++++++++++++++- .../spark/cypher/PropertyGraphReadWrite.scala | 18 +++++++++++++++++ .../spark/cypher/SharedCypherContext.scala | 18 +++++++++++++++++ .../construction/ScanGraphFactory.scala | 18 +++++++++++++++++ .../cypher/tck/SparkCypherTckSuite.scala | 18 +++++++++++++++++ 31 files changed, 542 insertions(+), 1 deletion(-) 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/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 index 80d7c9cd1fa3e..1719815eaa264 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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.sql.DataFrame 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 index 6a31bc14e4386..17ffea25736d7 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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.sql.{DataFrame, SaveMode, SparkSession, functions} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 0c562fe9629d5..b97b26cafe444 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -1,3 +1,21 @@ +/* + * 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.graph.api.GraphElementFrame.encodeIdColumns 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 index 2f9c5b17988d1..1b90a52fdebe6 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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.sql.{DataFrame, SaveMode} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala index 3a4ae25cffb81..4b3f18795175f 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala @@ -1,3 +1,21 @@ +/* + * 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 /** 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 index 4eee879acb0b0..7774cf7a4ca3e 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala @@ -1,3 +1,21 @@ +/* + * 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._ 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 index 343555f17e73a..f173016789e97 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala @@ -1,3 +1,21 @@ +/* + * 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._ 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 index 8699618a5083f..7f046e7022556 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index 9814c7be32f29..e69c4ff486cd0 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index 4dcbe1cdf5102..c8834054b07fd 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index 894694135c9fd..a0915b1a199d0 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index 2d92f6a6202ba..aea08f0a80831 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkGraphDirectoryStructure.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index bfe86ede9b5b3..fd3dcf7164325 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala @@ -1,3 +1,21 @@ +/* + * 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._ 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 index 06f6cc8754e75..fb6c64d96f1ce 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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.{NodeFrame, RelationshipFrame} 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 index 5497d3b6b2e95..569c1bdc28e64 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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 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 index 42aa7019b4406..58627299e3b94 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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.PropertyGraphType 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 index 4f123e98f26e0..21e3aed6bd690 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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} 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 index 2ccd37009e0e0..6a2d61c6d4f96 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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._ 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 index 5e5854ecee594..fbe8de2ed513b 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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._ 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 index ed0e8726072f9..c837211ae5e07 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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 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 index 19b07ecdd6e9e..126922ef05ad1 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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.time.temporal.ChronoUnit 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 index 43f1e07ab9b08..4977c4e32ad8d 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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._ 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 index 03f80c03eab35..06f55a4484839 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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 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 index 2b066f0c9ebd3..67a8df8db8d7b 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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} 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 index 6fc141394a609..a6668b1dd0343 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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} 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 index 6a68213511601..fc3d0cb1d9a30 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -1,3 +1,21 @@ +/* + * 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 @@ -72,7 +90,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val relsDF: DataFrame = spark.createDataFrame(Seq( (0L, 0L, 1L, true, false, Some(1984)), - (1L, 2L, 1L, false, true, None), + (1L, 2L, 1L, false, true, None) )).toDF("$ID", "$SOURCE_ID", "$TARGET_ID", ":KNOWS", ":TEACHES", "since") val graph: PropertyGraph = cypherSession.createGraph(nodeDF, relsDF) 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 index d3020110a15d9..ec03830a8bae2 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index 27013e52ac1d5..992058a2c053f 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/SharedCypherContext.scala @@ -1,3 +1,21 @@ +/* + * 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 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 index 4360dc6b667f8..114e2b01a477c 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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} 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 index 38957e27049ac..9901f1c3846fc 100644 --- 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 @@ -1,3 +1,21 @@ +/* + * 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 From 4abd883b66e7dae5b289751b3ed97dd77a2cd308 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 15 Apr 2019 11:37:28 -0400 Subject: [PATCH 072/123] Bump okapi dependencies to 0.3.2 --- graph/cypher/pom.xml | 22 --- .../spark/cypher/SparkCypherFunctions.scala | 12 +- .../cypher/conversions/ExprConversions.scala | 135 ++++++------------ .../cypher/conversions/RowConversion.scala | 19 +-- .../conversions/TemporalConversions.scala | 86 +++++++---- .../cypher/conversions/TypeConversions.scala | 51 ++----- .../src/test/resources/tck/failing_blacklist | 6 - 7 files changed, 132 insertions(+), 199 deletions(-) diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml index c35afb71f3909..35067a4d45118 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -45,13 +45,6 @@ org.apache.spark spark-sql_${scala.binary.version} ${project.version} - - - - org.antlr - antlr4-runtime - -
org.apache.spark @@ -80,13 +73,6 @@ ${project.version} test-jar test - - - - org.antlr - antlr4-runtime - - @@ -104,14 +90,6 @@ test - - - org.antlr - antlr4-runtime - 4.7.1 - test - - 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 index f173016789e97..d9bf1b20e23e8 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala @@ -24,7 +24,6 @@ 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, DataFrame, functions} -import org.opencypher.okapi.api.types.CypherType._ 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 @@ -54,7 +53,6 @@ object SparkCypherFunctions { 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 @@ -80,11 +78,15 @@ object SparkCypherFunctions { 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))) + 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))) + 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 = { @@ -95,7 +97,7 @@ object SparkCypherFunctions { 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))) + val transform = ArrayTransform(filtered, LambdaFunction(GetStructField(x, 0), Seq(x), hidden = false)) new Column(transform) } } 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 index 6a2d61c6d4f96..191b0645df556 100644 --- 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 @@ -29,41 +29,42 @@ import org.apache.spark.sql.{Column, DataFrame} import org.opencypher.okapi.api.types._ import org.opencypher.okapi.api.value.CypherValue.CypherMap import org.opencypher.okapi.impl.exception._ -import org.opencypher.okapi.impl.temporal.TemporalTypesHelper._ -import org.opencypher.okapi.impl.temporal.{Duration => DurationValue} -import org.opencypher.okapi.ir.api.PropertyKey import org.opencypher.okapi.ir.api.expr._ import org.opencypher.okapi.relational.impl.table.RecordHeader object ExprConversions { - /** - * 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, df: DataFrame, parameters: CypherMap): Column = { - if (expr.cypherType == CTNull) { - NULL_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)) + 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, df: DataFrame, parameters: CypherMap): Column = { + if (expr.cypherType == CTNull) { + NULL_LIT + } else if (expr.cypherType == CTTrue) { + TRUE_LIT + } else if (expr.cypherType == CTFalse) { + FALSE_LIT } else { - new Column(withConvertedChildrenResult) + 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) + } } } - } - - implicit class RichExpression(expr: Expr) { /** * Attempts to create a Spark SQL expression from the CAPS expression. @@ -131,60 +132,17 @@ object ExprConversions { case _ => explode(child0) } - case Property(e, PropertyKey(key)) => - // TODO: Convert property lookups into separate specific lookups instead of overloading - e.cypherType.material match { - case CTMap(inner) => if (inner.keySet.contains(key)) child0.getField(key) else NULL_LIT - case CTDate => temporalAccessor[java.sql.Date](child0, key) - case CTLocalDateTime => temporalAccessor[java.sql.Timestamp](child0, key) - case CTDuration => TemporalUdfs.durationAccessor(key.toLowerCase).apply(child0) - case _ => - if (!header.contains(expr)) { - NULL_LIT - } else { - column_for(expr) - } - } - case LocalDateTime(dateExpr) => - // TODO: Move code outside of expr mapper - dateExpr match { - case Some(e) => - val localDateTimeValue = resolveTemporalArgument(e) - .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 - - lit(localDateTimeValue).cast(DataTypes.TimestampType) - case None => current_timestamp() - } - - case Date(dateExpr) => - // TODO: Move code outside of expr mapper - dateExpr match { - case Some(e) => - val dateValue = resolveTemporalArgument(e) - .map(parseDate) - .map(java.sql.Date.valueOf) - .orNull - - lit(dateValue).cast(DataTypes.DateType) - case None => current_timestamp() - } + case _: EntityProperty => 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 Duration(durationExpr) => - // TODO: Move code outside of expr mapper - val durationValue = resolveTemporalArgument(durationExpr).map { - case Left(m) => DurationValue(m.mapValues(_.toLong)).toCalendarInterval - case Right(s) => DurationValue.parse(s).toCalendarInterval - }.orNull - lit(durationValue) + 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(CTVoid) => FALSE_LIT case CTList(inner) if inner.couldBeSameTypeAs(lhs.cypherType) => array_contains(child1, child0) case _ => NULL_LIT } @@ -195,13 +153,13 @@ object ExprConversions { val rhsCT = rhs.cypherType.material lhsCT -> rhsCT match { case (CTList(lhInner), CTList(rhInner)) => - if (lhInner.material == rhInner.material || lhInner == CTVoid || rhInner == CTVoid) { + 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 nonListType == inner.material || inner.material == CTVoid => concat(child0, array(child1)) - case (nonListType, CTList(inner)) if inner.material == nonListType || inner.material == CTVoid => concat(array(child0), child1) + 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) @@ -237,7 +195,7 @@ object ExprConversions { case Keys(e) => e.cypherType.material match { - case _: CTNode | _: CTRelationship => + case entity if entity.subTypeOf(CTEntity) => val possibleProperties = header.propertiesFor(e.owner.get).toSeq.sortBy(_.key.name) val propertyNames = possibleProperties.map(_.key.name) val propertyValues = possibleProperties.map(_.asSparkSQLExpr) @@ -256,7 +214,7 @@ object ExprConversions { case Properties(e) => e.cypherType.material match { - case _: CTNode | _: CTRelationship => + case entity if entity.subTypeOf(CTEntity) => val propertyExpressions = header.propertiesFor(e.owner.get).toSeq.sortBy(_.key.name) val propertyColumns = propertyExpressions .map(propertyExpression => propertyExpression.asSparkSQLExpr.as(propertyExpression.key.name)) @@ -336,17 +294,15 @@ object ExprConversions { 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 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 indexCol = index.asSparkSQLExpr val containerCol = container.asSparkSQLExpr - container.cypherType.material match { - case _: CTList | _: CTMap => containerCol.get(indexCol) + case c if c.subTypeOf(CTContainer) => containerCol.get(index.asSparkSQLExpr) case other => throw NotImplementedException(s"Accessing $other by index is not supported") } @@ -378,6 +334,9 @@ object ExprConversions { 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/RowConversion.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/RowConversion.scala index fbe8de2ed513b..8cdfda7bf510c 100644 --- 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 @@ -18,11 +18,9 @@ package org.apache.spark.cypher.conversions -import org.apache.spark.cypher.conversions.TemporalConversions._ import org.apache.spark.cypher.{SparkCypherNode, SparkCypherRelationship} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema -import org.apache.spark.unsafe.types.CalendarInterval import org.opencypher.okapi.api.types.{CTList, CTMap, CTNode, CTRelationship} import org.opencypher.okapi.api.value.CypherValue._ import org.opencypher.okapi.api.value._ @@ -43,15 +41,9 @@ final case class RowConversion(exprToColumn: Seq[(Expr, String)]) extends (Row = // 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 _: CTNode => - collectNode(row, v) - - case _: CTRelationship => - collectRel(row, v) - - case CTList(_) if !header.exprToColumn.contains(v) => - collectComplexList(row, v) - + 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) } } @@ -74,10 +66,7 @@ final case class RowConversion(exprToColumn: Seq[(Expr, String)]) extends (Row = case _ => val raw = row.getAs[Any](header.column(expr)) - raw match { - case interval: CalendarInterval => interval.toDuration - case other => CypherValue(other) - } + CypherValue(raw) } } 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 index 126922ef05ad1..015b33b8d307a 100644 --- 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 @@ -18,23 +18,28 @@ 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, NotImplementedException, UnsupportedOperationException} +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. */ @@ -56,13 +61,14 @@ object TemporalConversions { /** * 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 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, @@ -79,35 +85,63 @@ object TemporalConversions { } } - def resolveTemporalArgument(expr: Expr) - (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)) + implicit class TemporalExpression(val expr: Expr) extends AnyVal { - case Param(name) => - val s = parameters(name) match { - case CypherString(str) => str - case other => throw IllegalArgumentException(s"Parameter `$name` to be a CypherString", other) + 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 + } - Some(Right(s)) + def resolveDate(implicit parameters: CypherMap): Date = { + expr.resolveTemporalArgument + .map(parseDate) + .map(java.sql.Date.valueOf) + .orNull + } - case NullLit => None + 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 + } - case other => - throw NotImplementedException(s"Parsing temporal values is currently only supported for Literal-Maps and String literals, got $other") + 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 = { 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 index 4977c4e32ad8d..0854ba9c6d36b 100644 --- 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 @@ -31,22 +31,7 @@ import org.opencypher.okapi.relational.impl.table.RecordHeader object TypeConversions { - // Spark data types that are supported within the Cypher type system - val supportedTypes: Seq[DataType] = Seq( - // numeric - ByteType, - ShortType, - IntegerType, - LongType, - FloatType, - DoubleType, - // other - StringType, - BooleanType, - DateType, - TimestampType, - NullType - ) + val DEFAULT_PRECISION = 20 implicit class CypherTypeOps(val ct: CypherType) extends AnyVal { @@ -63,27 +48,22 @@ object TypeConversions { ct.material match { case CTString => Some(StringType) case CTInteger => Some(LongType) - case CTBoolean => Some(BooleanType) + 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 _: CTNode => Some(BinaryType) - case _: CTRelationship => Some(BinaryType) + case b if b.subTypeOf(CTBoolean) => Some(BooleanType) + case n if n.subTypeOf(CTEntity.nullable) => Some(BinaryType) // Spark uses String as the default array inner type - case CTList(CTVoid) => Some(ArrayType(StringType, containsNull = false)) + 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(CTNumber) => Some(ArrayType(DoubleType, containsNull = false)) - case CTList(CTNumber.nullable) => Some(ArrayType(DoubleType, containsNull = true)) - case CTList(elemType) => elemType.toSparkType.map(ArrayType(_, elemType.isNullable)) - case CTMap(inner) => - val innerFields = inner.map { - case (key, valueType) => valueType.toStructField(key) - }.toSeq - Some(StructType(innerFields)) - case _ => - None + 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 } } @@ -132,13 +112,14 @@ object TypeConversions { 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(CTList(CTVoid)) + case ArrayType(NullType, _) => Some(CTEmptyList) case BinaryType => Some(CTIdentity) case ArrayType(elemType, containsNull) => - elemType.toCypherType(containsNull).map(CTList) + 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 @@ -158,11 +139,7 @@ object TypeConversions { * * @return true, iff the data type is supported */ - def isCypherCompatible: Boolean = dt match { - case ArrayType(internalType, _) => internalType.isCypherCompatible - case StructType(fields) => fields.forall(_.dataType.isCypherCompatible) - case other => supportedTypes.contains(other) - } + def isCypherCompatible: Boolean = cypherCompatibleDataType.isDefined /** * Converts the given Spark data type into a Cypher type system compatible Spark data type. diff --git a/graph/cypher/src/test/resources/tck/failing_blacklist b/graph/cypher/src/test/resources/tck/failing_blacklist index bc3551537b015..ead08dd514a4c 100644 --- a/graph/cypher/src/test/resources/tck/failing_blacklist +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -200,18 +200,13 @@ Feature "Aggregation": Scenario "`min()` over strings" 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 types of LHS and RHS don't match - singleton list" 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 "IN should work with an empty list in the presence of other list elements: not matching" 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 "IN should return false when LHS contains a nested list and type mismatch on RHS - singleton version" Feature "ListOperations": Scenario "Equality between list and literal should return false" -Feature "ListOperations": Scenario "IN should return false when matching a list with a nested list with same elements" -Feature "ListOperations": Scenario "IN should return false when LHS contains a nested list and type mismatch on RHS" Feature "ListOperations": Scenario "IN should work with an empty list in the presence of other list elements: matching" Feature "ListOperations": Scenario "IN should return false if no match can be found, despite nulls" Feature "ListOperations": Scenario "Setting and returning the size of a list property" @@ -222,7 +217,6 @@ Feature "ListOperations": Scenario "IN should return true when LHS and RHS conta 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 false when types of LHS and RHS don't match - 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" From b69a416f15185ca75a5c9837df59830f16451a01 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 18 Apr 2019 15:08:16 -0400 Subject: [PATCH 073/123] Make CypherSession.save public --- .../main/scala/org/apache/spark/graph/api/CypherSession.scala | 2 +- .../main/scala/org/apache/spark/cypher/SparkCypherSession.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 index 17ffea25736d7..50e54dd426554 100644 --- 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 @@ -113,6 +113,6 @@ trait CypherSession { */ def load(path: String): PropertyGraph - private[spark] def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit + def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit } 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 index c8834054b07fd..64fa1f3ecdc66 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -82,7 +82,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) createGraph(graphImporter.nodeFrames, graphImporter.relationshipFrames) } - override private[spark] def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit = { + def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit = { val relationalGraph = toRelationalGraph(graph) val graphDirectoryStructure = SparkGraphDirectoryStructure(path) From 687980de8916cbcfff2b56ea681363b02d0408e5 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 21 May 2019 14:44:30 -0700 Subject: [PATCH 074/123] Bump okapi version to 0.4.0 --- .../org/apache/spark/cypher/SparkCypherEntity.scala | 6 +++--- .../org/apache/spark/cypher/SparkCypherRecords.scala | 4 ++-- .../org/apache/spark/cypher/SparkCypherSession.scala | 4 ++-- .../org/apache/spark/cypher/SparkEntityTable.scala | 10 +++++----- .../apache/spark/cypher/adapters/MappingAdapter.scala | 6 +++--- .../apache/spark/cypher/adapters/SchemaAdapter.scala | 4 ++-- .../spark/cypher/conversions/ExprConversions.scala | 10 +++++----- .../spark/cypher/conversions/TypeConversions.scala | 2 +- graph/cypher/src/test/resources/tck/failing_blacklist | 7 ------- .../spark/cypher/construction/ScanGraphFactory.scala | 4 ++-- 10 files changed, 25 insertions(+), 32 deletions(-) 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 index 7774cf7a4ca3e..02def3ee8158c 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherEntity.scala @@ -19,7 +19,7 @@ package org.apache.spark.cypher import org.apache.spark.cypher.SparkCypherEntity._ -import org.opencypher.okapi.api.value.CypherValue.{CypherMap, CypherNode, CypherRelationship} +import org.opencypher.okapi.api.value.CypherValue.{CypherMap, Node, Relationship} object SparkCypherEntity { @@ -34,7 +34,7 @@ case class SparkCypherNode( override val id: Seq[Byte], override val labels: Set[String] = Set.empty, override val properties: CypherMap = CypherMap.empty -) extends CypherNode[Seq[Byte]] { +) extends Node[Seq[Byte]] { override type I = SparkCypherNode @@ -55,7 +55,7 @@ case class SparkCypherRelationship( override val endId: Seq[Byte], override val relType: String, override val properties: CypherMap = CypherMap.empty -) extends CypherRelationship[Seq[Byte]] { +) extends Relationship[Seq[Byte]] { override type I = SparkCypherRelationship 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 index 7f046e7022556..d514cf15aafcb 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala @@ -27,7 +27,7 @@ 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.EntityTable +import org.opencypher.okapi.relational.api.io.ElementTable import org.opencypher.okapi.relational.api.table.{RelationalCypherRecords, RelationalCypherRecordsFactory} import org.opencypher.okapi.relational.impl.table._ @@ -48,7 +48,7 @@ case class SparkCypherRecordsFactory()(implicit caps: SparkCypherSession) extend SparkCypherRecords(initialHeader, initialDataFrame) } - override def fromEntityTable(entityTable: EntityTable[DataFrameTable]): SparkCypherRecords = { + override def fromElementTable(entityTable: ElementTable[DataFrameTable]): SparkCypherRecords = { SparkCypherRecords(entityTable.header, entityTable.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 index 64fa1f3ecdc66..aab79cee25ccd 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -27,7 +27,7 @@ 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.RelationalEntityTableFactory +import org.opencypher.okapi.relational.api.table.RelationalElementTableFactory object SparkCypherSession { def create(implicit sparkSession: SparkSession): CypherSession = new SparkCypherSession(sparkSession) @@ -56,7 +56,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) } } - override def entityTables: RelationalEntityTableFactory[DataFrameTable] = { + override def elementTables: RelationalElementTableFactory[DataFrameTable] = { throw UnsupportedOperationException("Graph construction with `CONSTRUCT` is not supported in Cypher 9") } 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 index a0915b1a199d0..f98889ac2469d 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkEntityTable.scala @@ -19,17 +19,17 @@ package org.apache.spark.cypher import org.apache.spark.cypher.SparkTable.DataFrameTable -import org.opencypher.okapi.api.io.conversion.EntityMapping -import org.opencypher.okapi.relational.api.io.EntityTable +import org.opencypher.okapi.api.io.conversion.ElementMapping +import org.opencypher.okapi.relational.api.io.ElementTable case class SparkEntityTable( - override val mapping: EntityMapping, + override val mapping: ElementMapping, override val table: DataFrameTable -) extends EntityTable[DataFrameTable] with RecordBehaviour { +) extends ElementTable[DataFrameTable] with RecordBehaviour { override type Records = SparkEntityTable - private[spark] def records(implicit cypherEngine: SparkCypherSession): SparkCypherRecords = cypherEngine.records.fromEntityTable(entityTable = this) + private[spark] def records(implicit cypherEngine: SparkCypherSession): SparkCypherRecords = cypherEngine.records.fromElementTable(entityTable = this) override def cache(): SparkEntityTable = { table.cache() 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 index fb6c64d96f1ce..2444eb5f07f10 100644 --- 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 @@ -19,12 +19,12 @@ package org.apache.spark.cypher.adapters import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} -import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMappingBuilder, RelationshipMappingBuilder} +import org.opencypher.okapi.api.io.conversion.{ElementMapping, NodeMappingBuilder, RelationshipMappingBuilder} object MappingAdapter { implicit class RichNodeDataFrame(val nodeDf: NodeFrame) extends AnyVal { - def toNodeMapping: EntityMapping = NodeMappingBuilder + def toNodeMapping: ElementMapping = NodeMappingBuilder .on(nodeDf.idColumn) .withImpliedLabels(nodeDf.labelSet.toSeq: _*) .withPropertyKeyMappings(nodeDf.properties.toSeq:_*) @@ -32,7 +32,7 @@ object MappingAdapter { } implicit class RichRelationshipDataFrame(val relDf: RelationshipFrame) extends AnyVal { - def toRelationshipMapping: EntityMapping = RelationshipMappingBuilder + def toRelationshipMapping: ElementMapping = RelationshipMappingBuilder .on(relDf.idColumn) .withSourceStartNodeKey(relDf.sourceIdColumn) .withSourceEndNodeKey(relDf.targetIdColumn) 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 index 58627299e3b94..5450c915d89dc 100644 --- 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 @@ -19,9 +19,9 @@ package org.apache.spark.cypher.adapters import org.apache.spark.graph.api.PropertyGraphType -import org.opencypher.okapi.api.schema.Schema +import org.opencypher.okapi.api.schema.PropertyGraphSchema -case class SchemaAdapter(schema: Schema) extends PropertyGraphType { +case class SchemaAdapter(schema: PropertyGraphSchema) extends PropertyGraphType { override def labelSets: Set[Set[String]] = schema.labelCombinations.combos 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 index 191b0645df556..d5346c0a3531d 100644 --- 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 @@ -132,7 +132,7 @@ object ExprConversions { case _ => explode(child0) } - case _: EntityProperty => if (!header.contains(expr)) NULL_LIT else column_for(expr) + 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) @@ -195,7 +195,7 @@ object ExprConversions { case Keys(e) => e.cypherType.material match { - case entity if entity.subTypeOf(CTEntity) => + 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) @@ -214,7 +214,7 @@ object ExprConversions { case Properties(e) => e.cypherType.material match { - case entity if entity.subTypeOf(CTEntity) => + 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)) @@ -264,9 +264,9 @@ object ExprConversions { case _: Atan => atan(child0) case _: Atan2 => atan2(child0, child1) case _: Cos => cos(child0) - case Cot(e) => Divide(IntegerLit(1), Tan(e))(CTFloat).asSparkSQLExpr + case Cot(e) => Divide(IntegerLit(1), Tan(e)).asSparkSQLExpr case _: Degrees => degrees(child0) - case Haversin(e) => Divide(Subtract(IntegerLit(1), Cos(e))(CTFloat), IntegerLit(2))(CTFloat).asSparkSQLExpr + case Haversin(e) => Divide(Subtract(IntegerLit(1), Cos(e)), IntegerLit(2)).asSparkSQLExpr case _: Radians => radians(child0) case _: Sin => sin(child0) case _: Tan => tan(child0) 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 index 0854ba9c6d36b..6d157a4af6655 100644 --- 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 @@ -55,7 +55,7 @@ object TypeConversions { case CTDuration => Some(CalendarIntervalType) case CTIdentity => Some(BinaryType) case b if b.subTypeOf(CTBoolean) => Some(BooleanType) - case n if n.subTypeOf(CTEntity.nullable) => Some(BinaryType) + 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)) diff --git a/graph/cypher/src/test/resources/tck/failing_blacklist b/graph/cypher/src/test/resources/tck/failing_blacklist index ead08dd514a4c..7661161bd429b 100644 --- a/graph/cypher/src/test/resources/tck/failing_blacklist +++ b/graph/cypher/src/test/resources/tck/failing_blacklist @@ -188,15 +188,9 @@ 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 "AggregationAcceptance": Scenario "`max()` should aggregate strings" -Feature "AggregationAcceptance": Scenario "`min()` should aggregate strings" Feature "Aggregation": Scenario "`min()` over mixed values" -Feature "Aggregation": Scenario "`min()` over list values" Feature "Aggregation": Scenario "`max()` over mixed values" Feature "Aggregation": Scenario "`max()` over mixed numeric values" -Feature "Aggregation": Scenario "`max()` over list values" -Feature "Aggregation": Scenario "`max()` over strings" -Feature "Aggregation": Scenario "`min()` over strings" 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" @@ -208,7 +202,6 @@ Feature "ListOperations": Scenario "IN should work with an empty list when compa 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 "IN should return false if no match can be found, despite nulls" 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" 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 index 114e2b01a477c..566684c0f9ad4 100644 --- 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 @@ -27,7 +27,7 @@ import org.apache.spark.cypher.{SparkCypherSession, SparkEntityTable} import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row} import org.opencypher.okapi.api.graph.Pattern -import org.opencypher.okapi.api.io.conversion.{EntityMapping, NodeMappingBuilder, RelationshipMappingBuilder} +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 @@ -39,7 +39,7 @@ import scala.collection.JavaConverters._ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { - def encodeIdColumns(df: DataFrame, mapping: EntityMapping): DataFrame = { + def encodeIdColumns(df: DataFrame, mapping: ElementMapping): DataFrame = { val idCols = mapping.allSourceIdKeys.map { columnName => val dataType = df.schema.fields(df.schema.fieldIndex(columnName)).dataType dataType match { From 8e7ef83551f35cc6501e8b6f9e99ee378347bd01 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 12 Jun 2019 13:34:17 +0200 Subject: [PATCH 075/123] Add PropertyGraphTest MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../spark/graph/api/CypherSession.scala | 7 + .../spark/cypher/PropertyGraphSuite.scala | 290 ++++++++++++++++++ 2 files changed, 297 insertions(+) create mode 100644 graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala 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 index 50e54dd426554..2d721b9e1bed8 100644 --- 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 @@ -20,6 +20,13 @@ package org.apache.spark.graph.api import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, functions} +object CypherSession { + val ID_COLUMN = "$ID" + val SOURCE_ID_COLUMN = "$SOURCE_ID" + val TARGET_ID_COLUMN = "$TARGET_ID" + val LABEL_COLUMN_PREFIX = ":" +} + /** * Allows for creating and loading [[PropertyGraph]] instances and running Cypher-queries on them. * Wraps a [[SparkSession]]. 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..8f2480cbdf99e --- /dev/null +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -0,0 +1,290 @@ +/* + * 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.graph.api.CypherSession._ +import org.apache.spark.graph.api.{CypherSession, NodeFrame, RelationshipFrame} +import org.apache.spark.sql.catalyst.util.NumberConverter +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.{DataFrame, QueryTest} +import org.scalatest.Matchers + +class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers { + + type IdType = Array[Byte] + + def convertId(inputId: Long): IdType = NumberConverter.toBinary(inputId) + + def cypherSession: CypherSession = SparkCypherSession.create + + test("create graph from NodeFrame") { + val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") + val nodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) + val graph = cypherSession.createGraph(Seq(nodeFrame), Seq.empty) + + 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 NodeFrame and RelationshipFrame") { + val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") + val nodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) + + val relationshipData = spark + .createDataFrame(Seq((0L, 0L, 1L, 1984))) + .toDF("id", "source", "target", "since") + val relationshipFrame = RelationshipFrame( + relationshipData, + idColumn = "id", + sourceIdColumn = "source", + targetIdColumn = "target", + relationshipType = "KNOWS") + + val graph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) + + 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 studentNF = + NodeFrame(initialDf = studentDF, idColumn = "id", labelSet = Set("Person", "Student")) + val teacherNF = + NodeFrame(initialDf = teacherDF, idColumn = "id", labelSet = Set("Person", "Teacher")) + + 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 knowsRF = RelationshipFrame( + initialDf = knowsDF, + idColumn = "id", + sourceIdColumn = "source", + targetIdColumn = "target", + relationshipType = "KNOWS") + val teachesRF = RelationshipFrame( + initialDf = teachesDF, + idColumn = "id", + sourceIdColumn = "source", + targetIdColumn = "target", + relationshipType = "TEACHES") + + val graph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + + 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 studentNF = NodeFrame( + initialDf = studentDF, + idColumn = "id", + labelSet = Set("Person", "Student"), + properties = Map("name" -> "col_name", "age" -> "col_age")) + val teacherNF = NodeFrame( + initialDf = teacherDF, + idColumn = "id", + labelSet = Set("Person", "Teacher"), + properties = 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 knowsRF = RelationshipFrame( + initialDf = knowsDF, + idColumn = "id", + sourceIdColumn = "source", + targetIdColumn = "target", + relationshipType = "KNOWS", + properties = Map("since" -> "col_since")) + val teachesRF = RelationshipFrame( + initialDf = teachesDF, + idColumn = "id", + sourceIdColumn = "source", + targetIdColumn = "target", + relationshipType = "TEACHES") + + val graph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + + 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) + } + + lazy val nodes: DataFrame = 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: DataFrame = 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("select nodes via label set") { + val graph = cypherSession.createGraph(nodes, relationships) + val nodeFrame = graph.nodeFrame(Set("Person", "Teacher")) + + nodeFrame.labelSet shouldEqual Set("Person", "Teacher") + nodeFrame.idColumn shouldEqual ID_COLUMN + nodeFrame.properties 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(nodeFrame.df, expectedNodeDf) + } + + test("select relationships via type") { + val graph = cypherSession.createGraph(nodes, relationships) + val relationshipFrame = graph.relationshipFrame("KNOWS") + + relationshipFrame.relationshipType shouldEqual "KNOWS" + relationshipFrame.idColumn shouldEqual ID_COLUMN + relationshipFrame.sourceIdColumn shouldEqual SOURCE_ID_COLUMN + relationshipFrame.targetIdColumn shouldEqual TARGET_ID_COLUMN + relationshipFrame.properties 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(relationshipFrame.df, expectedRelDf) + } + + private def label(label: String): String = s"$LABEL_COLUMN_PREFIX$label" + +} From c7608faff5061762c14ceb225fce97b0626e659e Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 12 Jun 2019 13:49:29 +0200 Subject: [PATCH 076/123] Bump okapi-shade 0.4.2 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../org/apache/spark/cypher/udfs/TemporalUdfs.scala | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) 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 index 67a8df8db8d7b..4ff30273342f3 100644 --- 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 @@ -21,7 +21,6 @@ package org.apache.spark.cypher.udfs import java.sql.{Date, Timestamp} import java.time.temporal.{ChronoField, IsoFields, TemporalField} -import org.apache.logging.log4j.scala.Logging import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions.udf import org.apache.spark.unsafe.types.CalendarInterval @@ -29,7 +28,7 @@ import org.opencypher.okapi.impl.exception.UnsupportedOperationException import scala.reflect.runtime.universe._ -object TemporalUdfs extends Logging { +object TemporalUdfs { /** * Adds a duration to a date. @@ -42,10 +41,6 @@ object TemporalUdfs extends Logging { } else { val days = interval.microseconds / CalendarInterval.MICROS_PER_DAY - if (interval.microseconds % CalendarInterval.MICROS_PER_DAY != 0) { - logger.warn("Arithmetic with Date and Duration can lead to incorrect results when sub-day values are present.") - } - val reducedLocalDate = date .toLocalDate .plusMonths(interval.months) @@ -66,10 +61,6 @@ object TemporalUdfs extends Logging { } else { val days = interval.microseconds / CalendarInterval.MICROS_PER_DAY - if (interval.microseconds % CalendarInterval.MICROS_PER_DAY != 0) { - logger.warn("Arithmetic with Date and Duration can lead to incorrect results when sub-day values are present.") - } - val reducedLocalDate = date .toLocalDate .minusMonths(interval.months) From 9cb99d64a03447635a661965349ab956e490bf0c Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 12 Jun 2019 14:11:30 +0200 Subject: [PATCH 077/123] Adapt GraphElementFrame API to latest API changes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../spark/graph/api/GraphElementFrame.scala | 295 ++++++++++-------- .../spark/cypher/SparkCypherSession.scala | 6 +- .../GraphElementFrameConversions.scala | 57 ++++ .../spark/cypher/GraphExamplesSuite.scala | 22 +- .../spark/cypher/PropertyGraphReadWrite.scala | 8 +- .../spark/cypher/PropertyGraphSuite.scala | 74 ++--- 6 files changed, 277 insertions(+), 185 deletions(-) create mode 100644 graph/cypher/src/main/scala/org/apache/spark/cypher/conversions/GraphElementFrameConversions.scala diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index b97b26cafe444..ad4acf0909af3 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -18,170 +18,205 @@ package org.apache.spark.graph.api -import org.apache.spark.graph.api.GraphElementFrame.encodeIdColumns import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.types._ - -object GraphElementFrame { - - def encodeIdColumns(df: DataFrame, idColumnNames: String*): DataFrame = { - val encodedIdCols = idColumnNames.map { idColumnName => - val col = df.col(idColumnName) - df.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 = df.columns.filterNot(idColumnNames.contains) - val remainingCols = remainingColumnNames.map(df.col) - df.select(encodedIdCols ++ remainingCols: _*) - } -} +import scala.collection.JavaConverters /** - * A [[PropertyGraph]] is created from GraphElementFrames. - * - * Wraps a [[DataFrame]] and describes how it maps to graph elements (i.e. nodes or relationships). - */ -trait GraphElementFrame { + * A [[PropertyGraph]] is created from GraphElementFrames. + * + * A graph element is either a node or a relationship. + * A GraphElementFrame wraps a [[DataFrame]] and describes how it maps to graph elements. + * + * @since 3.0.0 + */ +abstract class GraphElementFrame { /** - * Initial [[DataFrame]] that can still contain unmapped columns and ID columns that are not of type BinaryType. - * The columns are ordered arbitrarily. - */ - def initialDf: DataFrame + * Initial [[DataFrame]] that can still contain unmapped, arbitrarily ordered columns. + * + * @since 3.0.0 + */ + def df: DataFrame /** - * [[DataFrame]] that contains only mapped element data. Each row represents a graph element. - * Columns in 'initialDf' that do not have BinaryType are converted to BinaryType. - * ID columns are first, property columns are sorted alphabetically. - */ - val df: DataFrame = { - val mappedColumnNames = idColumns ++ properties.values.toSeq.sorted - val mappedDf = if (mappedColumnNames == initialDf.columns.toSeq) { - initialDf - } else { - initialDf.select(mappedColumnNames.map(initialDf.col): _*) - } - if (idColumns.forall(idColumn => initialDf.schema(idColumn).dataType == BinaryType)) { - mappedDf - } else { - encodeIdColumns(mappedDf, idColumns: _*) - } - } + * Name of the column that contains the graph element identifier. + * + * @since 3.0.0 + */ + def idColumn: String /** - * Name of the column that contains the graph element identifier. - * - * @note Column values need to be of [[org.apache.spark.sql.types.BinaryType]]. - */ - def idColumn: String + * Name of all columns that contain graph element identifiers. + * + * @since 3.0.0 + */ + def idColumns: Seq[String] = Seq(idColumn) /** - * Mapping from graph element property keys to the columns that contain the corresponding property values. - */ + * Mapping from graph element property keys to the columns that contain the corresponding property + * values. + * + * @since 3.0.0 + */ def properties: Map[String, String] - protected def idColumns: Seq[String] - } object NodeFrame { /** - * Describes how to map an input [[DataFrame]] to nodes. - * - * All columns apart from the given `idColumn` are mapped to node properties. - * - * @param initialDf [[DataFrame]] containing a single node in each row - * @param idColumn column that contains the node identifier - * @param labelSet labels that are assigned to all nodes - */ - def apply( - initialDf: DataFrame, - idColumn: String, - labelSet: Set[String] = Set.empty - ): NodeFrame = { - val properties = (initialDf.columns.toSet - idColumn) - .map(columnName => columnName -> columnName).toMap - NodeFrame(initialDf, idColumn, labelSet, properties) + * Describes how to map an initial [[DataFrame]] to nodes. + * + * All columns apart from the given `idColumn` are mapped to node properties. + * + * @param df [[DataFrame]] containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labelSet labels that are assigned to all nodes + * @since 3.0.0 + */ + def create(df: DataFrame, idColumn: String, labelSet: Set[String]): NodeFrame = { + val properties = (df.columns.toSet - idColumn) + .map(columnName => columnName -> columnName) + .toMap + + NodeFrame(df, idColumn, labelSet, properties) + } + + /** + * Describes how to map an initial [[DataFrame]] to nodes. + * + * All columns apart from the given `idColumn` are mapped to node properties. + * + * @param df [[DataFrame]] containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labelSet labels that are assigned to all nodes + * @since 3.0.0 + */ + def create(df: DataFrame, idColumn: String, labelSet: java.util.Set[String]): NodeFrame = { + create(df, idColumn, JavaConverters.asScalaSet(labelSet).toSet) + } + + /** + * Describes how to map an initial [[DataFrame]] to nodes. + * + * All columns apart from the given `idColumn` are mapped to node properties. + * + * @param df [[DataFrame]] 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def create( + df: DataFrame, + idColumn: String, + labelSet: java.util.Set[String], + properties: java.util.Map[String, String]): NodeFrame = { + val scalaLabelSet = JavaConverters.asScalaSet(labelSet).toSet + val scalaProperties = JavaConverters.mapAsScalaMap(properties).toMap + NodeFrame(df, idColumn, scalaLabelSet, scalaProperties) } } /** - * Describes how to map an input [[DataFrame]] to nodes. - * - * Each row in the [[DataFrame]] represents a node which has exactly the labels defined by the - * given label set. - * - * @param initialDf [[DataFrame]] 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 properties mapping from property keys to corresponding columns - */ + * Describes how to map a [[DataFrame]] to nodes. + * + * Each row in the [[DataFrame]] represents a node which has exactly the labels defined by the + * given label set. + * + * @param df [[DataFrame]] 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ case class NodeFrame( - initialDf: DataFrame, - idColumn: String, - labelSet: Set[String], - properties: Map[String, String] -) extends GraphElementFrame { - - override protected def idColumns: Seq[String] = Seq(idColumn) - -} + df: DataFrame, + idColumn: String, + labelSet: Set[String], + properties: Map[String, String]) + extends GraphElementFrame object RelationshipFrame { /** - * Describes how to map an input [[DataFrame]] to relationships. - * - * All columns apart from the given identifier columns are mapped to relationship properties. - * - * @param initialDf [[DataFrame]] 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 - */ - def apply( - initialDf: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String - ): RelationshipFrame = { - val properties = (initialDf.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) - .map(columnName => columnName -> columnName).toMap - RelationshipFrame(initialDf, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + * Describes how to map a [[DataFrame]] to relationships. + * + * All columns apart from the given identifier columns are mapped to relationship properties. + * + * @param df [[DataFrame]] 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 + * @since 3.0.0 + */ + def create( + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String): RelationshipFrame = { + val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) + .map(columnName => columnName -> columnName) + .toMap + + RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + } + + /** + * Describes how to map a [[DataFrame]] to relationships. + * + * @param df [[DataFrame]] 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def create( + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: java.util.Map[String, String]): RelationshipFrame = { + RelationshipFrame( + df, + idColumn, + sourceIdColumn, + targetIdColumn, + relationshipType, + JavaConverters.mapAsScalaMap(properties).toMap) } } /** - * Describes how to map an input [[DataFrame]] to relationships. - * - * Each row in the [[DataFrame]] represents a relationship with the given relationship type. - * - * @param initialDf [[DataFrame]] 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 properties mapping from property keys to corresponding columns - */ + * Describes how to map a [[DataFrame]] to relationships. + * + * Each row in the [[DataFrame]] represents a relationship with the given relationship type. + * + * @param df [[DataFrame]] 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ case class RelationshipFrame( - initialDf: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: Map[String, String] -) extends GraphElementFrame { - - override protected def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String]) + extends GraphElementFrame { + + override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) } 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 index aab79cee25ccd..3190647c44aec 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -20,6 +20,7 @@ package org.apache.spark.cypher 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.ReadWriteGraph._ import org.apache.spark.graph.api._ import org.apache.spark.sql.{SaveMode, SparkSession} @@ -65,7 +66,10 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) "There can be at most one NodeFrame per label set") require(relationships.groupBy(_.relationshipType).forall(_._2.size == 1), "There can be at most one RelationshipFrame per relationship type") - RelationalGraphAdapter(this, nodes, relationships) + + val normalizedNodes = nodes.map(nf => nf.copy(df = normalizeDf(nf))) + val normalizedRelationships = relationships.map(rf => rf.copy(df = normalizeDf(rf))) + RelationalGraphAdapter(this, normalizedNodes, normalizedRelationships) } def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) 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..aae7a5a85765c --- /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.GraphElementFrame +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{BinaryType, ByteType, IntegerType, LongType, ShortType, StringType} + +object GraphElementFrameConversions { + + def normalizeDf(frame: GraphElementFrame): DataFrame = { + val mappedColumnNames = frame.idColumns ++ frame.properties.values.toSeq.sorted + val mappedDf = if (mappedColumnNames == frame.df.columns.toSeq) { + frame.df + } else { + frame.df.select(mappedColumnNames.map(frame.df.col): _*) + } + if (frame.idColumns.forall(idColumn => frame.df.schema(idColumn).dataType == BinaryType)) { + mappedDf + } else { + encodeIdColumns(mappedDf, frame.idColumns: _*) + } + } + + private def encodeIdColumns(df: DataFrame, idColumnNames: String*): DataFrame = { + val encodedIdCols = idColumnNames.map { idColumnName => + val col = df.col(idColumnName) + df.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 = df.columns.filterNot(idColumnNames.contains) + val remainingCols = remainingColumnNames.map(df.col) + df.select(encodedIdCols ++ remainingCols: _*) + } + +} + 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 index fc3d0cb1d9a30..8d308511f119e 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -26,7 +26,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from single NodeFrame") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") - val nodeFrame: NodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) + val nodeFrame: NodeFrame = NodeFrame.create(df = nodeData, "id", Set("Person")) val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame)) val result: CypherResult = graph.cypher("MATCH (n) RETURN n") result.df.show() @@ -35,8 +35,8 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from Node- and RelationshipFrames") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") - val nodeFrame: NodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) - val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") + val nodeFrame: NodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) + val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) val result: CypherResult = graph.cypher( """ @@ -49,14 +49,14 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") - val studentNF: NodeFrame = NodeFrame(initialDf = studentDF, idColumn = "id", labelSet = Set("Person", "Student")) - val teacherNF: NodeFrame = NodeFrame(initialDf = teacherDF, idColumn = "id", labelSet = Set("Person", "Teacher")) + val studentNF: NodeFrame = NodeFrame.create(studentDF, "id", Set("Person", "Student")) + val teacherNF: NodeFrame = NodeFrame.create(teacherDF, "id", Set("Person", "Teacher")) val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") - val knowsRF: RelationshipFrame = RelationshipFrame(initialDf = knowsDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS") - val teachesRF: RelationshipFrame = RelationshipFrame(initialDf = teachesDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "TEACHES") + val knowsRF: RelationshipFrame = RelationshipFrame.create(knowsDF, "id", "source", "target", "KNOWS") + val teachesRF: RelationshipFrame = RelationshipFrame.create(teachesDF, "id", "source", "target", "TEACHES") val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") @@ -67,14 +67,14 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "col_name", "col_age") val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "col_name", "col_subject") - val studentNF: NodeFrame = NodeFrame(initialDf = studentDF, idColumn = "id", labelSet = Set("Person", "Student"), properties = Map("name" -> "col_name", "age" -> "col_age")) - val teacherNF: NodeFrame = NodeFrame(initialDf = teacherDF, idColumn = "id", labelSet = Set("Person", "Teacher"), properties = Map("name" -> "col_name", "subject" -> "col_subject")) + val studentNF: NodeFrame = NodeFrame(studentDF, "id", Set("Person", "Student"), Map("name" -> "col_name", "age" -> "col_age")) + val teacherNF: NodeFrame = NodeFrame(teacherDF, "id", Set("Person", "Teacher"), Map("name" -> "col_name", "subject" -> "col_subject")) val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "col_since") val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") - val knowsRF: RelationshipFrame = RelationshipFrame(initialDf = knowsDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS", properties = Map("since" -> "col_since")) - val teachesRF: RelationshipFrame = RelationshipFrame(initialDf = teachesDF, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "TEACHES") + val knowsRF: RelationshipFrame = RelationshipFrame(knowsDF, "id", "source", "target", "KNOWS", Map("since" -> "col_since")) + val teachesRF: RelationshipFrame = RelationshipFrame.create(teachesDF, "id", "source", "target", "TEACHES") val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) val result: CypherResult = graph.cypher("MATCH (n)-[r]->(m) RETURN n, r, m") 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 index ec03830a8bae2..2afb29709a817 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -52,13 +52,9 @@ class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with Tuple3(0, 0, 1) )).toDF("id", "source", "target") - private lazy val nodeDataFrame: NodeFrame = NodeFrame( - initialDf = nodeData, idColumn = "id", labelSet = Set("Person") - ) + private lazy val nodeDataFrame: NodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) - private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame( - relationshipData, idColumn = "id", sourceIdColumn = "source", targetIdColumn = "target", relationshipType = "KNOWS" - ) + private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") test("save and load a graph") { val graph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) 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 index 8f2480cbdf99e..828323006512c 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -35,7 +35,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeFrame") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) + val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) val graph = cypherSession.createGraph(Seq(nodeFrame), Seq.empty) val expectedDf = spark @@ -47,17 +47,17 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeFrame and RelationshipFrame") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = NodeFrame(initialDf = nodeData, idColumn = "id", labelSet = Set("Person")) + val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) .toDF("id", "source", "target", "since") - val relationshipFrame = RelationshipFrame( + val relationshipFrame = RelationshipFrame.create( relationshipData, - idColumn = "id", - sourceIdColumn = "source", - targetIdColumn = "target", - relationshipType = "KNOWS") + "id", + "source", + "target", + "KNOWS") val graph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) @@ -82,9 +82,9 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .toDF("id", "name", "subject") val studentNF = - NodeFrame(initialDf = studentDF, idColumn = "id", labelSet = Set("Person", "Student")) + NodeFrame.create(studentDF, "id", Set("Person", "Student")) val teacherNF = - NodeFrame(initialDf = teacherDF, idColumn = "id", labelSet = Set("Person", "Teacher")) + NodeFrame.create(teacherDF, "id", Set("Person", "Teacher")) val knowsDF = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) @@ -93,18 +93,18 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((1L, 2L, 1L))) .toDF("id", "source", "target") - val knowsRF = RelationshipFrame( - initialDf = knowsDF, - idColumn = "id", - sourceIdColumn = "source", - targetIdColumn = "target", - relationshipType = "KNOWS") - val teachesRF = RelationshipFrame( - initialDf = teachesDF, - idColumn = "id", - sourceIdColumn = "source", - targetIdColumn = "target", - relationshipType = "TEACHES") + val knowsRF = RelationshipFrame.create( + knowsDF, + "id", + "source", + "target", + "KNOWS") + val teachesRF = RelationshipFrame.create( + teachesDF, + "id", + "source", + "target", + "TEACHES") val graph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) @@ -150,32 +150,32 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .toDF("id", "col_name", "col_subject") val studentNF = NodeFrame( - initialDf = studentDF, - idColumn = "id", - labelSet = Set("Person", "Student"), + studentDF, + "id", + Set("Person", "Student"), properties = Map("name" -> "col_name", "age" -> "col_age")) val teacherNF = NodeFrame( - initialDf = teacherDF, - idColumn = "id", - labelSet = Set("Person", "Teacher"), + teacherDF, + "id", + Set("Person", "Teacher"), properties = 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 knowsRF = RelationshipFrame( - initialDf = knowsDF, - idColumn = "id", - sourceIdColumn = "source", - targetIdColumn = "target", + knowsDF, + "id", + "source", + "target", relationshipType = "KNOWS", properties = Map("since" -> "col_since")) - val teachesRF = RelationshipFrame( - initialDf = teachesDF, - idColumn = "id", - sourceIdColumn = "source", - targetIdColumn = "target", - relationshipType = "TEACHES") + val teachesRF = RelationshipFrame.create( + teachesDF, + "id", + "source", + "target", + "TEACHES") val graph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) From f985978bc6d9d9ddb86fc107289b5a5b7715b3be Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 24 Jun 2019 11:55:12 +0200 Subject: [PATCH 078/123] Add test for reading/writing Property Graphs --- .../spark/cypher/PropertyGraphReadWrite.scala | 25 ++++++++++++++++--- 1 file changed, 22 insertions(+), 3 deletions(-) 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 index 2afb29709a817..05f1d2b8000ca 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -20,13 +20,12 @@ package org.apache.spark.cypher import java.nio.file.Paths -import org.apache.spark.SparkFunSuite import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} import org.junit.rules.TemporaryFolder import org.scalatest.BeforeAndAfterEach -class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with BeforeAndAfterEach { +class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with BeforeAndAfterEach { private var tempDir: TemporaryFolder = _ @@ -66,4 +65,24 @@ class PropertyGraphReadWrite extends SparkFunSuite with SharedCypherContext with ).df.show() } + test("save and loads a property graph") { + val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") + val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) + + val relationshipData = spark + .createDataFrame(Seq((0L, 0L, 1L, 1984))) + .toDF("id", "source", "target", "since") + val relationshipFrame = + RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") + + val writeGraph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) + + withTempDir(file => { + cypherSession.save(writeGraph, file.getAbsolutePath, SaveMode.Overwrite) + val readGraph = cypherSession.load(file.getAbsolutePath) + + checkAnswer(readGraph.nodes, writeGraph.nodes) + checkAnswer(readGraph.relationships, writeGraph.relationships) + }) + } } From 52edea5fe0b8d2507482b6b600a51467d45f60a1 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 28 Jun 2019 09:20:43 +0200 Subject: [PATCH 079/123] Adapt to changes in SPIP PR --- .../org/apache/spark/graph/api/CypherSession.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) 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 index 2d721b9e1bed8..2f78d17a5a981 100644 --- 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 @@ -18,7 +18,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession, functions} +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} object CypherSession { val ID_COLUMN = "$ID" @@ -91,14 +91,11 @@ trait CypherSession { val labelColumns = nodes.columns.filter(_.startsWith(":")).toSet val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns).map(col => col -> col).toMap - val trueLit = functions.lit(true) - val falseLit = functions.lit(false) - val labelSets = labelColumns.subsets().toSet + Set.empty val nodeFrames = labelSets.map { labelSet => val predicate = labelColumns.map { - case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) === trueLit - case labelColumn => nodes.col(labelColumn) === falseLit + case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) + case labelColumn => !nodes.col(labelColumn) }.reduce(_ && _) NodeFrame(nodes.filter(predicate), idColumn, labelSet.map(_.substring(1)), nodeProperties) @@ -107,7 +104,7 @@ trait CypherSession { val relTypeColumns = relationships.columns.filter(_.startsWith(":")).toSet val relProperties = (relationships.columns.toSet - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns).map(col => col -> col).toMap val relFrames = relTypeColumns.map { relTypeColumn => - val predicate = relationships.col(relTypeColumn) === trueLit + val predicate = relationships.col(relTypeColumn) RelationshipFrame(relationships.filter(predicate), idColumn, sourceIdColumn, targetIdColumn, relTypeColumn.substring(1), relProperties) } From 23242cdfc5a3c92e1025a22b000673c0962c4d51 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 24 Jul 2019 21:59:31 +0200 Subject: [PATCH 080/123] Adapt to changes in https://github.com/apache/spark/pull/24851 --- .../apache/spark/graph/api/CypherResult.scala | 28 +-- .../spark/graph/api/CypherSession.scala | 180 ++++++++++++------ .../spark/graph/api/GraphElementFrame.scala | 160 ++++++++++------ .../spark/graph/api/PropertyGraph.scala | 149 +++++++++------ .../spark/graph/api/PropertyGraphType.scala | 27 ++- .../spark/cypher/SparkCypherSession.scala | 2 +- .../spark/cypher/GraphExamplesSuite.scala | 2 +- 7 files changed, 348 insertions(+), 200 deletions(-) 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 index 1719815eaa264..8b6049901435f 100644 --- 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 @@ -21,19 +21,23 @@ package org.apache.spark.graph.api import org.apache.spark.sql.DataFrame /** - * Result of a Cypher query. - * - * Wraps a [[DataFrame]] that contains the result rows. - */ + * Result of a Cypher query. + * + * Wraps a DataFrame that contains the result rows. + * + * @since 3.0.0 + */ trait CypherResult { + /** - * 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 the underscore (i.e. `_`), - * (e.g. `MATCH (n:Person) RETURN n` results in the columns `n`, `n:Person` and `n_name`). - */ + * 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 df: DataFrame } 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 index 2f78d17a5a981..6ffe591d5bc1f 100644 --- 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 @@ -18,6 +18,8 @@ package org.apache.spark.graph.api +import scala.collection.JavaConverters._ + import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} object CypherSession { @@ -28,95 +30,159 @@ object CypherSession { } /** - * Allows for creating and loading [[PropertyGraph]] instances and running Cypher-queries on them. - * Wraps a [[SparkSession]]. - */ + * The entry point for using property graphs in Spark. + * + * Provides factory methods for creating [[PropertyGraph]] instances. + * + * Wraps a [[org.apache.spark.sql.SparkSession]]. + * + * @since 3.0.0 + */ 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 - */ + * 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. - * - * @param graph [[PropertyGraph]] on which the query is executed - * @param query Cypher query to execute - * @param parameters parameters used by the Cypher query - */ + * Executes a Cypher query on the given input graph. + * + * @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 /** - * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. - * At least one [[NodeFrame]] has to be provided. - * - * For each label set and relationship type there can be ar most on [[NodeFrame]] and - * [[RelationshipFrame]], respectively. - * - * @param nodes [[NodeFrame]]s that define the nodes in the graph - * @param relationships [[RelationshipFrame]]s that define the relationships in the graph - */ - def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph + * Executes a Cypher query on the given input graph. + * + * @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 nodes and relationships. - * - * The given DataFrames need to adhere to 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) - * }}} - * - * @param nodes node [[DataFrame]] - * @param relationships relationship [[DataFrame]] - */ + * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. + * At least one [[NodeFrame]] has to be provided. + * + * For each label set and relationship type there can be at most one [[NodeFrame]] and at most one + * [[RelationshipFrame]], respectively. + * + * @param nodes NodeFrames that define the nodes in the graph + * @param relationships RelationshipFrames that define the relationships in the graph + * @since 3.0.0 + */ + def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph + + /** + * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. + * At least one [[NodeFrame]] has to be provided. + * + * For each label set and relationship type there can be at most one [[NodeFrame]] and at most one + * [[RelationshipFrame]], respectively. + * + * @param nodes NodeFrames that define the nodes in the graph + * @param relationships RelationshipFrames that define the relationships in the graph + * @since 3.0.0 + */ + def createGraph( + nodes: java.util.List[NodeFrame], + relationships: java.util.List[RelationshipFrame]): PropertyGraph = { + createGraph(nodes.asScala, relationships.asScala) + } + + /** + * Creates a [[PropertyGraph]] from nodes and relationships. + * + * The given DataFrames need 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) + * }}} + * + * @see [[CypherSession]] + * @param nodes node DataFrame + * @param relationships relationship DataFrame + * @since 3.0.0 + */ def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { - val idColumn = "$ID" - val sourceIdColumn = "$SOURCE_ID" - val targetIdColumn = "$TARGET_ID" + val idColumn = CypherSession.ID_COLUMN + val sourceIdColumn = CypherSession.SOURCE_ID_COLUMN + val targetIdColumn = CypherSession.TARGET_ID_COLUMN - val labelColumns = nodes.columns.filter(_.startsWith(":")).toSet - val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns).map(col => col -> col).toMap + val labelColumns = nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet + val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns) + .map(col => col -> col) + .toMap val labelSets = labelColumns.subsets().toSet + Set.empty val nodeFrames = labelSets.map { labelSet => - val predicate = labelColumns.map { - case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) - case labelColumn => !nodes.col(labelColumn) - }.reduce(_ && _) + val predicate = labelColumns + .map { + case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) + case labelColumn => !nodes.col(labelColumn) + } + .reduce(_ && _) NodeFrame(nodes.filter(predicate), idColumn, labelSet.map(_.substring(1)), nodeProperties) } - val relTypeColumns = relationships.columns.filter(_.startsWith(":")).toSet - val relProperties = (relationships.columns.toSet - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns).map(col => col -> col).toMap + val relColumns = relationships.columns.toSet + val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) + val propertyColumns = relColumns - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns + val relProperties = propertyColumns.map(col => col -> col).toMap val relFrames = relTypeColumns.map { relTypeColumn => val predicate = relationships.col(relTypeColumn) - RelationshipFrame(relationships.filter(predicate), idColumn, sourceIdColumn, targetIdColumn, relTypeColumn.substring(1), relProperties) + RelationshipFrame( + relationships.filter(predicate), + idColumn, + sourceIdColumn, + targetIdColumn, + relTypeColumn.substring(1), + relProperties) } createGraph(nodeFrames.toSeq, relFrames.toSeq) } /** - * Loads a [[PropertyGraph]] from the given location. - */ + * Loads a [[PropertyGraph]] from the given location. + * + * @param path directory in which the graph is stored + * @since 3.0.0 + */ def load(path: String): PropertyGraph + /** + * Saves a [[PropertyGraph]] to the given location. + * + * @param graph [[PropertyGraph]] to be stored + * @param path directory in which the graph should be stored + * @param saveMode specifies what happens when the destination already exists + * @since 3.0.0 + */ def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit - } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index ad4acf0909af3..145cda895bc4e 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -13,27 +13,26 @@ * 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.sql.DataFrame +import scala.collection.JavaConverters._ -import scala.collection.JavaConverters +import org.apache.spark.sql.DataFrame /** * A [[PropertyGraph]] is created from GraphElementFrames. * * A graph element is either a node or a relationship. - * A GraphElementFrame wraps a [[DataFrame]] and describes how it maps to graph elements. + * A GraphElementFrame wraps a DataFrame and describes how it maps to graph elements. * * @since 3.0.0 */ abstract class GraphElementFrame { /** - * Initial [[DataFrame]] that can still contain unmapped, arbitrarily ordered columns. + * Initial DataFrame that can still contain unmapped, arbitrarily ordered columns. * * @since 3.0.0 */ @@ -66,87 +65,105 @@ abstract class GraphElementFrame { object NodeFrame { /** - * Describes how to map an initial [[DataFrame]] to nodes. + * Describes how to map an initial DataFrame to nodes. * * All columns apart from the given `idColumn` are mapped to node properties. * - * @param df [[DataFrame]] 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 df DataFrame containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labelSet labels that are assigned to all nodes * @since 3.0.0 */ def create(df: DataFrame, idColumn: String, labelSet: Set[String]): NodeFrame = { val properties = (df.columns.toSet - idColumn) .map(columnName => columnName -> columnName) .toMap + create(df, idColumn, labelSet, properties) + } + /** + * Describes how to map an initial DataFrame to nodes. + * + * All columns apart from the given `idColumn` are mapped to node properties. + * + * @param df DataFrame 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def create( + df: DataFrame, + idColumn: String, + labelSet: Set[String], + properties: Map[String, String]): NodeFrame = { NodeFrame(df, idColumn, labelSet, properties) } /** - * Describes how to map an initial [[DataFrame]] to nodes. + * Describes how to map an initial DataFrame to nodes. * * All columns apart from the given `idColumn` are mapped to node properties. * - * @param df [[DataFrame]] 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 df DataFrame containing a single node in each row + * @param idColumn column that contains the node identifier + * @param labelSet labels that are assigned to all nodes * @since 3.0.0 */ def create(df: DataFrame, idColumn: String, labelSet: java.util.Set[String]): NodeFrame = { - create(df, idColumn, JavaConverters.asScalaSet(labelSet).toSet) + create(df, idColumn, labelSet.asScala.toSet) } /** - * Describes how to map an initial [[DataFrame]] to nodes. + * Describes how to map an initial DataFrame to nodes. * * All columns apart from the given `idColumn` are mapped to node properties. * - * @param df [[DataFrame]] 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 df DataFrame 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 properties mapping from property keys to corresponding columns * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - labelSet: java.util.Set[String], - properties: java.util.Map[String, String]): NodeFrame = { - val scalaLabelSet = JavaConverters.asScalaSet(labelSet).toSet - val scalaProperties = JavaConverters.mapAsScalaMap(properties).toMap + df: DataFrame, + idColumn: String, + labelSet: java.util.Set[String], + properties: java.util.Map[String, String]): NodeFrame = { + val scalaLabelSet = labelSet.asScala.toSet + val scalaProperties = properties.asScala.toMap NodeFrame(df, idColumn, scalaLabelSet, scalaProperties) } } /** - * Describes how to map a [[DataFrame]] to nodes. + * Describes how to map a DataFrame to nodes. * - * Each row in the [[DataFrame]] represents a node which has exactly the labels defined by the - * given label set. + * Each row in the DataFrame represents a node which has exactly the labels defined by the given + * label set. * - * @param df [[DataFrame]] containing a single node in each row + * @param df DataFrame 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 properties mapping from property keys to corresponding columns * @since 3.0.0 */ -case class NodeFrame( - df: DataFrame, - idColumn: String, - labelSet: Set[String], - properties: Map[String, String]) - extends GraphElementFrame +case class NodeFrame private[graph] ( + df: DataFrame, + idColumn: String, + labelSet: Set[String], + properties: Map[String, String]) + extends GraphElementFrame object RelationshipFrame { /** - * Describes how to map a [[DataFrame]] to relationships. + * Describes how to map a DataFrame to relationships. * * All columns apart from the given identifier columns are mapped to relationship properties. * - * @param df [[DataFrame]] containing a single relationship in each row + * @param df DataFrame 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 @@ -154,22 +171,43 @@ object RelationshipFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String): RelationshipFrame = { + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String): RelationshipFrame = { val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) .map(columnName => columnName -> columnName) .toMap + create(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + } + + /** + * Describes how to map a DataFrame to relationships. + * + * @param df DataFrame 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def create( + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String]): RelationshipFrame = { RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) } /** - * Describes how to map a [[DataFrame]] to relationships. + * Describes how to map a DataFrame to relationships. * - * @param df [[DataFrame]] containing a single relationship in each row + * @param df DataFrame 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 @@ -178,29 +216,29 @@ object RelationshipFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: java.util.Map[String, String]): RelationshipFrame = { + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: java.util.Map[String, String]): RelationshipFrame = { RelationshipFrame( df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, - JavaConverters.mapAsScalaMap(properties).toMap) + properties.asScala.toMap) } } /** - * Describes how to map a [[DataFrame]] to relationships. + * Describes how to map a DataFrame to relationships. * - * Each row in the [[DataFrame]] represents a relationship with the given relationship type. + * Each row in the DataFrame represents a relationship with the given relationship type. * - * @param df [[DataFrame]] containing a single relationship in each row + * @param df DataFrame 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 @@ -208,14 +246,14 @@ object RelationshipFrame { * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ -case class RelationshipFrame( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: Map[String, String]) - extends GraphElementFrame { +case class RelationshipFrame private[graph] ( + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String]) + extends GraphElementFrame { override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) 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 index 1b90a52fdebe6..039c649ab1e46 100644 --- 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 @@ -13,7 +13,6 @@ * 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 @@ -21,87 +20,121 @@ package org.apache.spark.graph.api import org.apache.spark.sql.{DataFrame, SaveMode} /** - * A Property Graph as defined by the openCypher Property Graph Model. - * - * A graph is always tied to and managed by a [[CypherSession]]. The lifetime of a graph is bounded by the session lifetime. - * - * @see [[https://github.com/opencypher/openCypher/blob/master/docs/property-graph-model.adoc openCypher Property Graph Model]] - */ -trait PropertyGraph { + * 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 + * @since 3.0.0 + */ +abstract class PropertyGraph { /** - * The schema (graph type) describes the structure of this graph. - */ + * The schema (graph type) describes the structure of this graph. + * + * @since 3.0.0 + */ def schema: PropertyGraphType /** - * The session in which this graph is managed. - */ + * 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 - */ - def cypher(query: String): CypherResult = cypher(query, Map.empty) + * 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 - */ - def cypher(query: String, parameters: Map[String, Any]): CypherResult = cypherSession.cypher(this, query, parameters) + * 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) /** - * Returns the [[NodeFrame]] for a given node label set. - * - * @param labelSet Label set used for [[NodeFrame]] lookup - * @return [[NodeFrame]] for the given label set - */ + * 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 [[NodeFrame]] for a given node label set. + * + * @param labelSet Label set used for NodeFrame lookup + * @return NodeFrame for the given label set + * @since 3.0.0 + */ def nodeFrame(labelSet: Set[String]): NodeFrame /** - * Returns the [[RelationshipFrame]] for a given relationship type. - * - * @param relationshipType Relationship type used for [[RelationshipFrame]] lookup - * @return [[RelationshipFrame]] for the given relationship type - */ + * Returns the [[RelationshipFrame]] for a given relationship type. + * + * @param relationshipType Relationship type used for RelationshipFrame lookup + * @return RelationshipFrame for the given relationship type + * @since 3.0.0 + */ def relationshipFrame(relationshipType: String): RelationshipFrame /** - * Returns a [[DataFrame]] that contains a row for each node in this graph. - * - * The DataFrame adheres to column naming conventions: - * - * {{{ - * Id column: `$ID` - * Label columns: `:{LABEL_NAME}` - * Property columns: `{Property_Key}` - * }}} - */ + * Returns a DataFrame that contains a row for each node in this graph. + * + * The DataFrame 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: DataFrame /** - * Returns a [[DataFrame]] that contains a row for each relationship in this graph. - * - * The DataFrame adheres to column naming conventions: - * - * {{{ - * Id column: `$ID` - * SourceId column: `$SOURCE_ID` - * TargetId column: `$TARGET_ID` - * RelType columns: `:{REL_TYPE}` - * Property columns: `{Property_Key}` - * }}} - */ + * Returns a DataFrame that contains a row for each relationship in this + * graph. + * + * The DataFrame 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: DataFrame /** - * Saves this graph to the given location. - */ + * Saves this graph to the given location. + * + * @param path directory in which the graph should be stored + * @param saveMode specifies what happens when the destination already exists + * @since 3.0.0 + */ def save(path: String, saveMode: SaveMode = SaveMode.ErrorIfExists): Unit = cypherSession.save(this, path, saveMode) - } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala index 4b3f18795175f..1aabd23a882ef 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala @@ -13,28 +13,35 @@ * 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 /** - * Describes the structure of a [[PropertyGraph]]. - */ + * Describes the structure of a [[PropertyGraph]]. + * + * @since 3.0.0 + */ trait PropertyGraphType { + /** - * Returns all labels occurring on any node in the graph. - */ + * Returns all labels occurring on any node in the graph. + * + * @since 3.0.0 + */ def labels: Set[String] = labelSets.flatten /** - * Returns all distinct label sets occurring on nodes in the graph. - */ + * Returns all distinct label sets occurring on nodes in the graph. + * + * @since 3.0.0 + */ def labelSets: Set[Set[String]] /** - * Returns all relationship types occurring on relationships in the graph. - */ + * Returns all relationship types occurring on relationships in the graph. + * + * @since 3.0.0 + */ def relationshipTypes: Set[String] - } 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 index 3190647c44aec..9ae0f86c01b9d 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -72,7 +72,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) RelationalGraphAdapter(this, normalizedNodes, normalizedRelationships) } - def cypher(graph: PropertyGraph, query: String): CypherResult = cypher(graph, query, Map.empty) + 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) 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 index 8d308511f119e..2f3099ec50fe1 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -27,7 +27,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from single NodeFrame") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeFrame: NodeFrame = NodeFrame.create(df = nodeData, "id", Set("Person")) - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame), Seq.empty) val result: CypherResult = graph.cypher("MATCH (n) RETURN n") result.df.show() } From bee92a3c86eaa9d2b8d7972f3d919ddfb0c708c5 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 12:25:35 +0200 Subject: [PATCH 081/123] Warn if label columns > 5 and throw if label columns > 10 --- .../apache/spark/graph/api/CypherSession.scala | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) 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 index 6ffe591d5bc1f..ffad94930dce4 100644 --- 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 @@ -20,6 +20,7 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} object CypherSession { @@ -38,7 +39,7 @@ object CypherSession { * * @since 3.0.0 */ -trait CypherSession { +trait CypherSession extends Logging { def sparkSession: SparkSession @@ -137,7 +138,17 @@ trait CypherSession { .map(col => col -> col) .toMap - val labelSets = labelColumns.subsets().toSet + Set.empty + val labelCount = labelColumns.size + if (labelCount > 5) { + log.warn(s"$labelCount label columns will result in ${Math.pow(labelCount, 2)} node frames.") + 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 + val nodeFrames = labelSets.map { labelSet => val predicate = labelColumns .map { From 5b7e5e1b39c6b5b132895567698e15301aa38b0a Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 12:33:52 +0200 Subject: [PATCH 082/123] Add note and fix indentation --- .../apache/spark/graph/api/CypherSession.scala | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) 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 index ffad94930dce4..058604881c00c 100644 --- 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 @@ -13,7 +13,6 @@ * 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 @@ -71,9 +70,9 @@ trait CypherSession extends Logging { * @since 3.0.0 */ def cypher( - graph: PropertyGraph, - query: String, - parameters: java.util.Map[String, Object]): CypherResult = { + graph: PropertyGraph, + query: String, + parameters: java.util.Map[String, Object]): CypherResult = { cypher(graph, query, parameters.asScala.toMap) } @@ -102,8 +101,8 @@ trait CypherSession extends Logging { * @since 3.0.0 */ def createGraph( - nodes: java.util.List[NodeFrame], - relationships: java.util.List[RelationshipFrame]): PropertyGraph = { + nodes: java.util.List[NodeFrame], + relationships: java.util.List[RelationshipFrame]): PropertyGraph = { createGraph(nodes.asScala, relationships.asScala) } @@ -123,6 +122,9 @@ trait CypherSession extends Logging { * Property columns: `{Property_Key}` (nodes and relationships) * }}} * + * @note It is recommended to cache the input DataFrames if they represent multiple label sets and + * relationship types. + * * @see [[CypherSession]] * @param nodes node DataFrame * @param relationships relationship DataFrame @@ -140,7 +142,8 @@ trait CypherSession extends Logging { val labelCount = labelColumns.size if (labelCount > 5) { - log.warn(s"$labelCount label columns will result in ${Math.pow(labelCount, 2)} node frames.") + log.warn( + s"$labelCount label columns will result in ${Math.pow(labelCount, 2)} node frames.") if (labelCount > 10) { throw new IllegalArgumentException( s"Expected number of label columns to be less than or equal to 10, was $labelCount.") From 7cfe8278a6d2c48b59ecc2a1713180f9459f2888 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 13:22:10 +0200 Subject: [PATCH 083/123] Add inline doc that explains reasoning behind having CypherResult --- .../main/scala/org/apache/spark/graph/api/CypherResult.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 index 8b6049901435f..cee3042a5b87f 100644 --- 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 @@ -13,7 +13,6 @@ * 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 @@ -28,6 +27,8 @@ import org.apache.spark.sql.DataFrame * @since 3.0.0 */ 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. From 777c65218555724403000cad2721a2d2c2c98403 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 13:35:45 +0200 Subject: [PATCH 084/123] Add Java test for PropertyGraph creation --- graph/api/pom.xml | 28 ++++++ .../graph/api/JavaPropertyGraphSuite.java | 99 +++++++++++++++++++ graph/cypher/pom.xml | 8 ++ .../api/SparkCypherPropertyGraphSuite.java | 28 ++++++ 4 files changed, 163 insertions(+) create mode 100644 graph/api/src/test/java/org/apache/spark/graph/api/JavaPropertyGraphSuite.java create mode 100644 graph/cypher/src/test/java/org/apache/spark/graph/api/SparkCypherPropertyGraphSuite.java 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/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..24f3c5b4dbca2 --- /dev/null +++ b/graph/api/src/test/java/org/apache/spark/graph/api/JavaPropertyGraphSuite.java @@ -0,0 +1,99 @@ +/* + * 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 com.google.common.collect.Sets; +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; + + 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 testCreateFromNodeFrame() { + 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); + NodeFrame personNodeFrame = NodeFrame + .create(personDf, "id", Sets.newHashSet("Person")); + + Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); + RelationshipFrame knowsRelFrame = RelationshipFrame + .create(knowsDf, "id", "source", "target", "KNOWS"); + + PropertyGraph graph = cypherSession.createGraph( + Lists.newArrayList(personNodeFrame), + Lists.newArrayList(knowsRelFrame)); + 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 35067a4d45118..d6cee744c2bab 100644 --- a/graph/cypher/pom.xml +++ b/graph/cypher/pom.xml @@ -59,6 +59,14 @@ + + org.apache.spark + spark-graph-api_${scala.binary.version} + ${project.version} + test-jar + test + + org.apache.spark spark-core_${scala.binary.version} 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); + } +} From b36868641fe2a17a3862a0992887c611d6e4c294 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 13:58:05 +0200 Subject: [PATCH 085/123] Add doc to CypherSession companion object --- .../apache/spark/graph/api/CypherSession.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) 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 index 058604881c00c..9353367597c7d 100644 --- 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 @@ -22,10 +22,25 @@ import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +/** + * Contains constants used for convention based column naming. + */ 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 both for node label and relationship type prefixes. + */ val LABEL_COLUMN_PREFIX = ":" } From a1200c2923aea8d19ce99b2116776e817333b897 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 14:08:32 +0200 Subject: [PATCH 086/123] Update documentation for CypherSession trait --- .../scala/org/apache/spark/graph/api/CypherSession.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 index 9353367597c7d..675cf60d9a9e5 100644 --- 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 @@ -45,9 +45,8 @@ object CypherSession { } /** - * The entry point for using property graphs in Spark. - * - * Provides factory methods for creating [[PropertyGraph]] instances. + * 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]]. * From ca950a223e0d25cced175bea9985cd72e4c5f85f Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 14:16:28 +0200 Subject: [PATCH 087/123] Add reference to Neo4j Cypher manual on CypherSession::cypher methods --- .../main/scala/org/apache/spark/graph/api/CypherSession.scala | 3 +++ 1 file changed, 3 insertions(+) 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 index 675cf60d9a9e5..9fbfb49c7b232 100644 --- 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 @@ -59,6 +59,7 @@ trait CypherSession extends Logging { /** * Executes a Cypher query on the given input graph. * + * @see * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @since 3.0.0 @@ -68,6 +69,7 @@ trait CypherSession extends Logging { /** * Executes a Cypher query on the given input graph. * + * @see * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query @@ -78,6 +80,7 @@ trait CypherSession extends Logging { /** * Executes a Cypher query on the given input graph. * + * @see * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query From 82e5ad7f4592ea101cec2a6a5c106120d4146ec2 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 14:21:24 +0200 Subject: [PATCH 088/123] Add reference to Neo4j Cypher parameter docs on CypherSession::cypher --- .../main/scala/org/apache/spark/graph/api/CypherSession.scala | 3 +++ 1 file changed, 3 insertions(+) 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 index 9fbfb49c7b232..5a1f098b91cdc 100644 --- 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 @@ -60,6 +60,7 @@ trait CypherSession extends Logging { * Executes a Cypher query on the given input graph. * * @see + * @see * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @since 3.0.0 @@ -70,6 +71,7 @@ trait CypherSession extends Logging { * Executes a Cypher query on the given input graph. * * @see + * @see * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query @@ -81,6 +83,7 @@ trait CypherSession extends Logging { * Executes a Cypher query on the given input graph. * * @see + * @see * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query From eb6985ba64b1075fc880f9ecaf541b9b07881084 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 14:32:48 +0200 Subject: [PATCH 089/123] Add parameter example to CypherSession::cypher methods --- .../spark/graph/api/CypherSession.scala | 53 +++++++++++++++---- 1 file changed, 44 insertions(+), 9 deletions(-) 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 index 5a1f098b91cdc..fb962f6b2aec9 100644 --- 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 @@ -26,18 +26,22 @@ import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} * Contains constants used for convention based column naming. */ 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 both for node label and relationship type prefixes. */ @@ -70,6 +74,22 @@ trait CypherSession extends Logging { /** * 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 + * }}} + * * @see * @see * @param graph [[PropertyGraph]] on which the query is executed @@ -82,6 +102,22 @@ trait CypherSession extends Logging { /** * 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 + * }}} + * * @see * @see * @param graph [[PropertyGraph]] on which the query is executed @@ -90,9 +126,9 @@ trait CypherSession extends Logging { * @since 3.0.0 */ def cypher( - graph: PropertyGraph, - query: String, - parameters: java.util.Map[String, Object]): CypherResult = { + graph: PropertyGraph, + query: String, + parameters: java.util.Map[String, Object]): CypherResult = { cypher(graph, query, parameters.asScala.toMap) } @@ -121,8 +157,8 @@ trait CypherSession extends Logging { * @since 3.0.0 */ def createGraph( - nodes: java.util.List[NodeFrame], - relationships: java.util.List[RelationshipFrame]): PropertyGraph = { + nodes: java.util.List[NodeFrame], + relationships: java.util.List[RelationshipFrame]): PropertyGraph = { createGraph(nodes.asScala, relationships.asScala) } @@ -144,7 +180,6 @@ trait CypherSession extends Logging { * * @note It is recommended to cache the input DataFrames if they represent multiple label sets and * relationship types. - * * @see [[CypherSession]] * @param nodes node DataFrame * @param relationships relationship DataFrame @@ -213,9 +248,9 @@ trait CypherSession extends Logging { /** * Saves a [[PropertyGraph]] to the given location. * - * @param graph [[PropertyGraph]] to be stored - * @param path directory in which the graph should be stored - * @param saveMode specifies what happens when the destination already exists + * @param graph [[PropertyGraph]] to be stored + * @param path directory in which the graph should be stored + * @param saveMode specifies what happens when the destination already exists * @since 3.0.0 */ def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit From 671e680077bcecde882bb15caaa588c1306b8026 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 14:55:15 +0200 Subject: [PATCH 090/123] Accept Arrays instead of Seq of Node-/RelationshipFrames --- .../spark/graph/api/CypherSession.scala | 21 ++----------------- .../graph/api/JavaPropertyGraphSuite.java | 4 ++-- .../spark/cypher/SparkCypherSession.scala | 8 +++---- .../spark/cypher/GraphExamplesSuite.scala | 8 +++---- .../spark/cypher/PropertyGraphReadWrite.scala | 4 ++-- .../spark/cypher/PropertyGraphSuite.scala | 12 +++++------ 6 files changed, 20 insertions(+), 37 deletions(-) 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 index fb962f6b2aec9..3b30d49137c2d 100644 --- 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 @@ -143,24 +143,7 @@ trait CypherSession extends Logging { * @param relationships RelationshipFrames that define the relationships in the graph * @since 3.0.0 */ - def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame]): PropertyGraph - - /** - * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. - * At least one [[NodeFrame]] has to be provided. - * - * For each label set and relationship type there can be at most one [[NodeFrame]] and at most one - * [[RelationshipFrame]], respectively. - * - * @param nodes NodeFrames that define the nodes in the graph - * @param relationships RelationshipFrames that define the relationships in the graph - * @since 3.0.0 - */ - def createGraph( - nodes: java.util.List[NodeFrame], - relationships: java.util.List[RelationshipFrame]): PropertyGraph = { - createGraph(nodes.asScala, relationships.asScala) - } + def createGraph(nodes: Array[NodeFrame], relationships: Array[RelationshipFrame]): PropertyGraph /** * Creates a [[PropertyGraph]] from nodes and relationships. @@ -234,7 +217,7 @@ trait CypherSession extends Logging { relProperties) } - createGraph(nodeFrames.toSeq, relFrames.toSeq) + createGraph(nodeFrames.toArray, relFrames.toArray) } /** 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 index 24f3c5b4dbca2..b481f51df158e 100644 --- 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 @@ -83,8 +83,8 @@ public void testCreateFromNodeFrame() { .create(knowsDf, "id", "source", "target", "KNOWS"); PropertyGraph graph = cypherSession.createGraph( - Lists.newArrayList(personNodeFrame), - Lists.newArrayList(knowsRelFrame)); + new NodeFrame[]{personNodeFrame}, + new RelationshipFrame[]{knowsRelFrame}); List result = graph.nodes().collectAsList(); Assert.assertEquals(2, result.size()); } 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 index 9ae0f86c01b9d..6637ca0ef9346 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -61,10 +61,10 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) throw UnsupportedOperationException("Graph construction with `CONSTRUCT` is not supported in Cypher 9") } - override def createGraph(nodes: Seq[NodeFrame], relationships: Seq[RelationshipFrame] = Seq.empty): PropertyGraph = { - require(nodes.groupBy(_.labelSet).forall(_._2.size == 1), + override def createGraph(nodes: Array[NodeFrame], relationships: Array[RelationshipFrame]): PropertyGraph = { + require(nodes.groupBy(_.labelSet).forall(_._2.length == 1), "There can be at most one NodeFrame per label set") - require(relationships.groupBy(_.relationshipType).forall(_._2.size == 1), + require(relationships.groupBy(_.relationshipType).forall(_._2.length == 1), "There can be at most one RelationshipFrame per relationship type") val normalizedNodes = nodes.map(nf => nf.copy(df = normalizeDf(nf))) @@ -83,7 +83,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) override def load(path: String): PropertyGraph = { val graphImporter = GraphImporter(sparkSession, path, DEFAULT_FORMAT) - createGraph(graphImporter.nodeFrames, graphImporter.relationshipFrames) + createGraph(graphImporter.nodeFrames.toArray, graphImporter.relationshipFrames.toArray) } def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit = { 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 index 2f3099ec50fe1..9ceb7feae131c 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -27,7 +27,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from single NodeFrame") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeFrame: NodeFrame = NodeFrame.create(df = nodeData, "id", Set("Person")) - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame), Seq.empty) + val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) val result: CypherResult = graph.cypher("MATCH (n) RETURN n") result.df.show() } @@ -37,7 +37,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") val nodeFrame: NodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") - val graph: PropertyGraph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) val result: CypherResult = graph.cypher( """ |MATCH (a:Person)-[r:KNOWS]->(:Person) @@ -58,7 +58,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val knowsRF: RelationshipFrame = RelationshipFrame.create(knowsDF, "id", "source", "target", "KNOWS") val teachesRF: RelationshipFrame = RelationshipFrame.create(teachesDF, "id", "source", "target", "TEACHES") - val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + 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.df.show() } @@ -76,7 +76,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val knowsRF: RelationshipFrame = RelationshipFrame(knowsDF, "id", "source", "target", "KNOWS", Map("since" -> "col_since")) val teachesRF: RelationshipFrame = RelationshipFrame.create(teachesDF, "id", "source", "target", "TEACHES") - val graph: PropertyGraph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + 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.df.show() } 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 index 05f1d2b8000ca..7012f3974a862 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -56,7 +56,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") test("save and load a graph") { - val graph = cypherSession.createGraph(Seq(nodeDataFrame), Seq(relationshipFrame)) + val graph = cypherSession.createGraph(Array(nodeDataFrame), Array(relationshipFrame)) graph.save(basePath) val readGraph = cypherSession.load(basePath) @@ -75,7 +75,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef val relationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") - val writeGraph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) + val writeGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) withTempDir(file => { cypherSession.save(writeGraph, file.getAbsolutePath, SaveMode.Overwrite) 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 index 828323006512c..51bcb6df0ac86 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -36,7 +36,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeFrame") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) - val graph = cypherSession.createGraph(Seq(nodeFrame), Seq.empty) + val graph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) val expectedDf = spark .createDataFrame(Seq((convertId(0L), true, "Alice"), (convertId(1L), true, "Bob"))) @@ -59,7 +59,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "target", "KNOWS") - val graph = cypherSession.createGraph(Seq(nodeFrame), Seq(relationshipFrame)) + val graph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) val expectedNodeDf = spark .createDataFrame(Seq((convertId(0L), true, "Alice"), (convertId(1L), true, "Bob"))) @@ -106,14 +106,14 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "target", "TEACHES") - val graph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + val graph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) 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")), + (convertId(2L), true, false, true, None, Some("Eve"), Some("CS")) )) .toDF( ID_COLUMN, @@ -177,14 +177,14 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "target", "TEACHES") - val graph = cypherSession.createGraph(Seq(studentNF, teacherNF), Seq(knowsRF, teachesRF)) + val graph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) 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")), + (convertId(2L), true, false, true, None, Some("Eve"), Some("CS")) )) .toDF( ID_COLUMN, From 8fa15c40061a90c07116a1c33bdeca9f66274312 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 15:12:21 +0200 Subject: [PATCH 091/123] Change from case to if statement to improve readability --- .../scala/org/apache/spark/graph/api/CypherSession.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) 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 index 3b30d49137c2d..dfa05b731c7e6 100644 --- 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 @@ -192,9 +192,12 @@ trait CypherSession extends Logging { val nodeFrames = labelSets.map { labelSet => val predicate = labelColumns - .map { - case labelColumn if labelSet.contains(labelColumn) => nodes.col(labelColumn) - case labelColumn => !nodes.col(labelColumn) + .map { labelColumn => + if (labelSet.contains(labelColumn)) { + nodes.col(labelColumn) + } else { + !nodes.col(labelColumn) + } } .reduce(_ && _) From c77f399f00dc1de171cd77ce3f74e202f2f0ea91 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 16:09:57 +0200 Subject: [PATCH 092/123] Validate that label columns have BooleanType --- .../org/apache/spark/graph/api/CypherSession.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) 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 index dfa05b731c7e6..a7aa9a4ba761b 100644 --- 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 @@ -18,9 +18,9 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ - import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +import org.apache.spark.sql.types.{BooleanType, StructType} /** * Contains constants used for convention based column naming. @@ -169,11 +169,21 @@ trait CypherSession extends Logging { * @since 3.0.0 */ def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { + def validateLabelColumns(schema: StructType, columns: Set[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.") + } + }) + } + val idColumn = CypherSession.ID_COLUMN val sourceIdColumn = CypherSession.SOURCE_ID_COLUMN val targetIdColumn = CypherSession.TARGET_ID_COLUMN val labelColumns = nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet + validateLabelColumns(nodes.schema, labelColumns) + val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns) .map(col => col -> col) .toMap @@ -206,6 +216,7 @@ trait CypherSession extends Logging { val relColumns = relationships.columns.toSet val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) + validateLabelColumns(relationships.schema, relTypeColumns) val propertyColumns = relColumns - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns val relProperties = propertyColumns.map(col => col -> col).toMap val relFrames = relTypeColumns.map { relTypeColumn => From 4444260d7652879c3a5629b4e5b91578f094212d Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 25 Jul 2019 18:08:14 +0200 Subject: [PATCH 093/123] Make GraphElementFrame sealed --- .../spark/graph/api/GraphElementFrame.scala | 76 +++++++++---------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 145cda895bc4e..71562eced17d7 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.DataFrame * * @since 3.0.0 */ -abstract class GraphElementFrame { +sealed abstract class GraphElementFrame { /** * Initial DataFrame that can still contain unmapped, arbitrarily ordered columns. @@ -93,10 +93,10 @@ object NodeFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - labelSet: Set[String], - properties: Map[String, String]): NodeFrame = { + df: DataFrame, + idColumn: String, + labelSet: Set[String], + properties: Map[String, String]): NodeFrame = { NodeFrame(df, idColumn, labelSet, properties) } @@ -126,10 +126,10 @@ object NodeFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - labelSet: java.util.Set[String], - properties: java.util.Map[String, String]): NodeFrame = { + df: DataFrame, + idColumn: String, + labelSet: java.util.Set[String], + properties: java.util.Map[String, String]): NodeFrame = { val scalaLabelSet = labelSet.asScala.toSet val scalaProperties = properties.asScala.toMap NodeFrame(df, idColumn, scalaLabelSet, scalaProperties) @@ -150,11 +150,11 @@ object NodeFrame { * @since 3.0.0 */ case class NodeFrame private[graph] ( - df: DataFrame, - idColumn: String, - labelSet: Set[String], - properties: Map[String, String]) - extends GraphElementFrame + df: DataFrame, + idColumn: String, + labelSet: Set[String], + properties: Map[String, String]) + extends GraphElementFrame object RelationshipFrame { @@ -171,11 +171,11 @@ object RelationshipFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String): RelationshipFrame = { + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String): RelationshipFrame = { val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) .map(columnName => columnName -> columnName) .toMap @@ -195,12 +195,12 @@ object RelationshipFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: Map[String, String]): RelationshipFrame = { + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String]): RelationshipFrame = { RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) } @@ -216,12 +216,12 @@ object RelationshipFrame { * @since 3.0.0 */ def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: java.util.Map[String, String]): RelationshipFrame = { + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: java.util.Map[String, String]): RelationshipFrame = { RelationshipFrame( df, idColumn, @@ -247,13 +247,13 @@ object RelationshipFrame { * @since 3.0.0 */ case class RelationshipFrame private[graph] ( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: Map[String, String]) - extends GraphElementFrame { + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String]) + extends GraphElementFrame { override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) From e63c0cd13135f307a466923239c63e6eefae3dcf Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 26 Jul 2019 09:21:52 +0200 Subject: [PATCH 094/123] Add link to Property Graph Model to the class doc --- .../main/scala/org/apache/spark/graph/api/PropertyGraph.scala | 1 + 1 file changed, 1 insertion(+) 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 index 039c649ab1e46..5c3e30af24493 100644 --- 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 @@ -26,6 +26,7 @@ import org.apache.spark.sql.{DataFrame, SaveMode} * The lifetime of a graph is bound by the session lifetime. * * @see openCypher project + * @see PG Model * @since 3.0.0 */ abstract class PropertyGraph { From 0a938d49dbaddde4bf3b1cf9150e2f4def943f04 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 26 Jul 2019 10:11:45 +0200 Subject: [PATCH 095/123] Change PropertyGraph::labelSet to accept Array instead of Set --- .../main/scala/org/apache/spark/graph/api/PropertyGraph.scala | 2 +- .../apache/spark/cypher/adapters/RelationalGraphAdapter.scala | 2 +- .../test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) 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 index 5c3e30af24493..219d746f08b13 100644 --- 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 @@ -83,7 +83,7 @@ abstract class PropertyGraph { * @return NodeFrame for the given label set * @since 3.0.0 */ - def nodeFrame(labelSet: Set[String]): NodeFrame + def nodeFrame(labelSet: Array[String]): NodeFrame /** * Returns the [[RelationshipFrame]] for a given relationship type. 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 index 569c1bdc28e64..db9fc7d605c85 100644 --- 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 @@ -83,7 +83,7 @@ case class RelationalGraphAdapter( df.select(selectColumns: _*) } - override def nodeFrame(labelSet: Set[String]): NodeFrame = _nodeFrame(labelSet) + override def nodeFrame(labelSet: Array[String]): NodeFrame = _nodeFrame(labelSet.toSet) override def relationshipFrame(relationshipType: String): RelationshipFrame = _relationshipFrame(relationshipType) 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 index 51bcb6df0ac86..4164ae05ff1a3 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -249,7 +249,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("select nodes via label set") { val graph = cypherSession.createGraph(nodes, relationships) - val nodeFrame = graph.nodeFrame(Set("Person", "Teacher")) + val nodeFrame = graph.nodeFrame(Array("Person", "Teacher")) nodeFrame.labelSet shouldEqual Set("Person", "Teacher") nodeFrame.idColumn shouldEqual ID_COLUMN From ea2bf1b78ba370b3daa7bfa6ac9c738539ae5207 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 26 Jul 2019 13:05:07 +0200 Subject: [PATCH 096/123] Sync docs with API PR --- .../spark/graph/api/CypherSession.scala | 32 +++++++++++-------- .../spark/graph/api/PropertyGraph.scala | 2 +- 2 files changed, 19 insertions(+), 15 deletions(-) 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 index a7aa9a4ba761b..b5191ee35f1c7 100644 --- 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 @@ -18,6 +18,7 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ + import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} import org.apache.spark.sql.types.{BooleanType, StructType} @@ -63,8 +64,7 @@ trait CypherSession extends Logging { /** * Executes a Cypher query on the given input graph. * - * @see - * @see + * @see Cypher Manual * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @since 3.0.0 @@ -90,8 +90,8 @@ trait CypherSession extends Logging { * RETURN n * }}} * - * @see - * @see + * @see Cypher Manual + * @see Parameters * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query @@ -118,8 +118,8 @@ trait CypherSession extends Logging { * RETURN n * }}} * - * @see - * @see + * @see Cypher Manual + * @see Parameters * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query @@ -163,6 +163,7 @@ trait CypherSession extends Logging { * * @note It is recommended to cache the input DataFrames if they represent multiple label sets and * relationship types. + * * @see [[CypherSession]] * @param nodes node DataFrame * @param relationships relationship DataFrame @@ -170,11 +171,14 @@ trait CypherSession extends Logging { */ def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { def validateLabelColumns(schema: StructType, columns: Set[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.") - } - }) + 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.") + } + }) } val idColumn = CypherSession.ID_COLUMN @@ -245,9 +249,9 @@ trait CypherSession extends Logging { /** * Saves a [[PropertyGraph]] to the given location. * - * @param graph [[PropertyGraph]] to be stored - * @param path directory in which the graph should be stored - * @param saveMode specifies what happens when the destination already exists + * @param graph [[PropertyGraph]] to be stored + * @param path directory in which the graph should be stored + * @param saveMode specifies what happens when the destination already exists * @since 3.0.0 */ def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit 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 index 219d746f08b13..a42560c090921 100644 --- 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 @@ -26,7 +26,7 @@ import org.apache.spark.sql.{DataFrame, SaveMode} * The lifetime of a graph is bound by the session lifetime. * * @see openCypher project - * @see PG Model + * @see Property Graph Model * @since 3.0.0 */ abstract class PropertyGraph { From 714d1493d9926b28eac6d66f78b06f4becebe372 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 8 Oct 2019 15:27:33 +0200 Subject: [PATCH 097/123] Fix modules in parent pom Co-authored-by: Mats Rydberg --- pom.xml | 2 -- 1 file changed, 2 deletions(-) diff --git a/pom.xml b/pom.xml index e5c1e80d3826c..5b6994ca6d722 100644 --- a/pom.xml +++ b/pom.xml @@ -97,8 +97,6 @@ sql/catalyst sql/core sql/hive - graph/api - graph/cypher assembly examples repl From 94d73551eb08ec776a689f758e0e6e7c3465d712 Mon Sep 17 00:00:00 2001 From: Mats Rydberg Date: Tue, 8 Oct 2019 17:00:11 +0200 Subject: [PATCH 098/123] Restructure save/load API to match DataFrameWriter/Reader Co-authored-by: Martin Junghanns --- .../spark/graph/api/CypherSession.scala | 14 +-- .../spark/graph/api/PropertyGraph.scala | 9 +- .../spark/graph/api/PropertyGraphReader.scala | 42 +++++++++ .../spark/graph/api/PropertyGraphWriter.scala | 92 +++++++++++++++++++ .../spark/cypher/SparkCypherSession.scala | 53 +++-------- .../adapters/RelationalGraphAdapter.scala | 4 + .../io/SparkCypherPropertyGraphReader.scala | 31 +++++++ .../io/SparkCypherPropertyGraphWriter.scala | 48 ++++++++++ .../spark/cypher/GraphExamplesSuite.scala | 4 +- .../spark/cypher/PropertyGraphReadWrite.scala | 8 +- 10 files changed, 243 insertions(+), 62 deletions(-) create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphReader.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphWriter.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphReader.scala create mode 100644 graph/cypher/src/main/scala/org/apache/spark/cypher/io/SparkCypherPropertyGraphWriter.scala 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 index b5191ee35f1c7..bb215ce8a09af 100644 --- 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 @@ -239,20 +239,10 @@ trait CypherSession extends Logging { } /** - * Loads a [[PropertyGraph]] from the given location. + * Returns a [[PropertyGraphReader]] that can be used to read data in as a `PropertyGraph`. * - * @param path directory in which the graph is stored * @since 3.0.0 */ - def load(path: String): PropertyGraph + def read: PropertyGraphReader - /** - * Saves a [[PropertyGraph]] to the given location. - * - * @param graph [[PropertyGraph]] to be stored - * @param path directory in which the graph should be stored - * @param saveMode specifies what happens when the destination already exists - * @since 3.0.0 - */ - def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit } 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 index a42560c090921..8d15b7e731755 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.DataFrame /** * A Property Graph as defined by the openCypher Property Graph Data Model. @@ -130,12 +130,9 @@ abstract class PropertyGraph { def relationships: DataFrame /** - * Saves this graph to the given location. + * Interface for saving this `PropertyGraph` out into external storage. * - * @param path directory in which the graph should be stored - * @param saveMode specifies what happens when the destination already exists * @since 3.0.0 */ - def save(path: String, saveMode: SaveMode = SaveMode.ErrorIfExists): Unit = - cypherSession.save(this, path, saveMode) + 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..760f8fc9d769e --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphReader.scala @@ -0,0 +1,42 @@ +/* + * 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 + +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/PropertyGraphWriter.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphWriter.scala new file mode 100644 index 0000000000000..528cf5a737088 --- /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.sql.SaveMode + +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: + *
    + *
  • `SaveMode.Overwrite`: overwrite the existing data.
  • + *
  • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
  • + *
  • `SaveMode.ErrorIfExists`: throw an exception at runtime.
  • + *
+ *

+ * 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: + *

    + *
  • `overwrite`: overwrite the existing graph.
  • + *
  • `ignore`: ignore the operation (i.e. no-op).
  • + *
  • `error` or `errorifexists`: default option, throw an exception at runtime.
  • + *
+ * + * @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" => mode(SaveMode.ErrorIfExists) + case "default" => this + case "append" => mode(SaveMode.Append) + 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/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala index 6637ca0ef9346..2a3579502981d 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -18,12 +18,13 @@ 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.ReadWriteGraph._ +import org.apache.spark.cypher.io.SparkCypherPropertyGraphReader import org.apache.spark.graph.api._ -import org.apache.spark.sql.{SaveMode, SparkSession} +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} @@ -34,6 +35,16 @@ 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 + ) + } + } } /** @@ -79,41 +90,7 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) SparkCypherResult(relationalGraph.cypher(query, CypherMap(parameters.toSeq: _*)).records) } - private val DEFAULT_FORMAT = "parquet" - - override def load(path: String): PropertyGraph = { - val graphImporter = GraphImporter(sparkSession, path, DEFAULT_FORMAT) - createGraph(graphImporter.nodeFrames.toArray, graphImporter.relationshipFrames.toArray) - } - - def save(graph: PropertyGraph, path: String, saveMode: SaveMode): Unit = { - val relationalGraph = toRelationalGraph(graph) - val graphDirectoryStructure = SparkGraphDirectoryStructure(path) - - relationalGraph.schema.labelCombinations.combos.foreach { combo => - relationalGraph.canonicalNodeTable(combo) - .write - .format(DEFAULT_FORMAT) - .mode(saveMode) - .save(graphDirectoryStructure.pathToNodeTable(combo)) - } - relationalGraph.schema.relationshipTypes.foreach { relType => - relationalGraph.canonicalRelationshipTable(relType) - .write - .format(DEFAULT_FORMAT) - .mode(saveMode) - .save(graphDirectoryStructure.pathToRelationshipTable(relType)) - } - } - - private 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 - ) - } - } + override def read(): SparkCypherPropertyGraphReader = + new SparkCypherPropertyGraphReader(this) } 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 index db9fc7d605c85..95f7ebca87c42 100644 --- 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 @@ -20,6 +20,7 @@ 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.{NodeFrame, PropertyGraph, PropertyGraphType, RelationshipFrame} import org.apache.spark.sql.DataFrame @@ -87,4 +88,7 @@ case class RelationalGraphAdapter( override def relationshipFrame(relationshipType: String): RelationshipFrame = _relationshipFrame(relationshipType) + override def write(): SparkCypherPropertyGraphWriter = + + new SparkCypherPropertyGraphWriter(this) } 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..01c64232ced7c --- /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.nodeFrames.toArray, graphImporter.relationshipFrames.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/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala index 9ceb7feae131c..bc9b80c449a05 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -101,8 +101,8 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("save and load PropertyGraph") { val graph1: PropertyGraph = cypherSession.createGraph(nodes, relationships) graph1.nodes.show() - graph1.save("/tmp/my-storage", SaveMode.Overwrite) - val graph2: PropertyGraph = cypherSession.load("/tmp/my-storage") + graph1.write.mode(SaveMode.Overwrite).save("/tmp/my-storage") + val graph2: PropertyGraph = cypherSession.read.load("/tmp/my-storage") graph2.nodes.show() } 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 index 7012f3974a862..c91d2d79621d8 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -57,9 +57,9 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef test("save and load a graph") { val graph = cypherSession.createGraph(Array(nodeDataFrame), Array(relationshipFrame)) - graph.save(basePath) + graph.write.save(basePath) - val readGraph = cypherSession.load(basePath) + val readGraph = cypherSession.read.load(basePath) readGraph.cypher( "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" ).df.show() @@ -78,8 +78,8 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef val writeGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) withTempDir(file => { - cypherSession.save(writeGraph, file.getAbsolutePath, SaveMode.Overwrite) - val readGraph = cypherSession.load(file.getAbsolutePath) + 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) From 8ac4c3f87365020e47305c73d97490d0072395cf Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 9 Oct 2019 12:24:26 +0200 Subject: [PATCH 099/123] Replace NodeFrame object with NodeFrameBuilder MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../spark/graph/api/CypherSession.scala | 9 ++ .../spark/graph/api/GraphElementFrame.scala | 87 ++++++++----------- .../graph/api/JavaPropertyGraphSuite.java | 8 +- .../spark/cypher/GraphExamplesSuite.scala | 25 +++++- .../spark/cypher/PropertyGraphReadWrite.scala | 12 ++- .../spark/cypher/PropertyGraphSuite.scala | 28 ++++-- 6 files changed, 103 insertions(+), 66 deletions(-) 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 index bb215ce8a09af..0aca517d85fba 100644 --- 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 @@ -245,4 +245,13 @@ trait CypherSession extends Logging { */ def read: PropertyGraphReader + /** + * Returns a [[NodeFrameBuilder]] that can be used to construct a [[NodeFrame]]. + * + * @param df DataFrame containing a single node in each row + * @since 3.0.0 + */ + def buildNodeFrame(df: DataFrame): NodeFrameBuilder = + new NodeFrameBuilder(df) + } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 71562eced17d7..20bb1de30df5c 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -62,77 +62,64 @@ sealed abstract class GraphElementFrame { } -object NodeFrame { +/** + * Interface used to build a [[NodeFrame]]. + * + * @param dataFrame DataFrame containing a single node in each row + * @since 3.0.0 + */ +final class NodeFrameBuilder(var dataFrame: DataFrame) { + + private var idColumn: String = "id" + private var labelSet: Set[String] = Set.empty + private var properties: Map[String, String] = Map.empty /** - * Describes how to map an initial DataFrame to nodes. - * - * All columns apart from the given `idColumn` are mapped to node properties. - * - * @param df DataFrame 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 idColumn column that contains the node identifier * @since 3.0.0 */ - def create(df: DataFrame, idColumn: String, labelSet: Set[String]): NodeFrame = { - val properties = (df.columns.toSet - idColumn) - .map(columnName => columnName -> columnName) - .toMap - create(df, idColumn, labelSet, properties) + def idColumn(idColumn: String): NodeFrameBuilder = { + 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]): NodeFrameBuilder = { + this.labelSet = labelSet.toSet + this } /** - * Describes how to map an initial DataFrame to nodes. - * - * All columns apart from the given `idColumn` are mapped to node properties. - * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def create( - df: DataFrame, - idColumn: String, - labelSet: Set[String], - properties: Map[String, String]): NodeFrame = { - NodeFrame(df, idColumn, labelSet, properties) + def properties(properties: Map[String, String]): NodeFrameBuilder = { + this.properties = properties + this } /** - * Describes how to map an initial DataFrame to nodes. - * - * All columns apart from the given `idColumn` are mapped to node properties. - * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def create(df: DataFrame, idColumn: String, labelSet: java.util.Set[String]): NodeFrame = { - create(df, idColumn, labelSet.asScala.toSet) + def properties(properties: java.util.Map[String, String]): NodeFrameBuilder = { + this.properties = properties.asScala.toMap + this } /** - * Describes how to map an initial DataFrame to nodes. - * - * All columns apart from the given `idColumn` are mapped to node properties. + * Creates a `NodeFrame` from the specified builder parameters. * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def create( - df: DataFrame, - idColumn: String, - labelSet: java.util.Set[String], - properties: java.util.Map[String, String]): NodeFrame = { - val scalaLabelSet = labelSet.asScala.toSet - val scalaProperties = properties.asScala.toMap - NodeFrame(df, idColumn, scalaLabelSet, scalaProperties) + def build(): NodeFrame = { + NodeFrame(dataFrame, idColumn, labelSet, properties) } } 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 index b481f51df158e..0690703c755c4 100644 --- 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 @@ -18,7 +18,6 @@ package org.apache.spark.graph.api; import com.google.common.collect.Lists; -import com.google.common.collect.Sets; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -75,8 +74,11 @@ public void testCreateFromNodeFrame() { List knowsData = Collections.singletonList(RowFactory.create(0L, 0L, 1L, 1984)); Dataset personDf = spark.createDataFrame(personData, personSchema); - NodeFrame personNodeFrame = NodeFrame - .create(personDf, "id", Sets.newHashSet("Person")); + NodeFrame personNodeFrame = cypherSession.buildNodeFrame(personDf) + .idColumn("id") + .labelSet(new String[]{"Person"}) + .properties(Collections.singletonMap("name", "name")) + .build(); Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); RelationshipFrame knowsRelFrame = RelationshipFrame 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 index bc9b80c449a05..58cb400bff91f 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -26,7 +26,11 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from single NodeFrame") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") - val nodeFrame: NodeFrame = NodeFrame.create(df = nodeData, "id", Set("Person")) + val nodeFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) val result: CypherResult = graph.cypher("MATCH (n) RETURN n") result.df.show() @@ -35,7 +39,11 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from Node- and RelationshipFrames") { val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") - val nodeFrame: NodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) + val nodeFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) val result: CypherResult = graph.cypher( @@ -49,8 +57,17 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") - val studentNF: NodeFrame = NodeFrame.create(studentDF, "id", Set("Person", "Student")) - val teacherNF: NodeFrame = NodeFrame.create(teacherDF, "id", Set("Person", "Teacher")) + val studentNF = cypherSession.buildNodeFrame(studentDF) + .idColumn("id") + .labelSet(Array("Person", "Student")) + .properties(Map("name" -> "name", "age" -> "age")) + .build() + + val teacherNF = cypherSession.buildNodeFrame(teacherDF) + .idColumn("id") + .labelSet(Array("Person", "Teacher")) + .properties(Map("name" -> "name", "subject" -> "subject")) + .build() val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") 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 index c91d2d79621d8..95011181e2298 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -51,7 +51,11 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef Tuple3(0, 0, 1) )).toDF("id", "source", "target") - private lazy val nodeDataFrame: NodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) + private lazy val nodeDataFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") @@ -67,7 +71,11 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef test("save and loads a property graph") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) + val nodeFrame = cypherSession.buildNodeFrame(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) 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 index 4164ae05ff1a3..d0c359e4ec3eb 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -35,7 +35,11 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeFrame") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) + val nodeFrame = cypherSession.buildNodeFrame(nodeData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() val graph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) val expectedDf = spark @@ -47,8 +51,11 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeFrame and RelationshipFrame") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = NodeFrame.create(nodeData, "id", Set("Person")) - + val nodeFrame = cypherSession.buildNodeFrame(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") @@ -81,10 +88,17 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((2L, "Eve", "CS"))) .toDF("id", "name", "subject") - val studentNF = - NodeFrame.create(studentDF, "id", Set("Person", "Student")) - val teacherNF = - NodeFrame.create(teacherDF, "id", Set("Person", "Teacher")) + val studentNF = cypherSession.buildNodeFrame(studentDF) + .idColumn("id") + .labelSet(Array("Person", "Student")) + .properties(Map("name" -> "name", "age" -> "age")) + .build() + + val teacherNF = cypherSession.buildNodeFrame(teacherDF) + .idColumn("id") + .labelSet(Array("Person", "Teacher")) + .properties(Map("name" -> "name", "subject" -> "subject")) + .build() val knowsDF = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) From 5b2d08a21101408c49d0a4bbf371922eebeb78b0 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Wed, 9 Oct 2019 15:40:39 +0200 Subject: [PATCH 100/123] Replace RelationshipFrame object with RelationshipFrameBuilder --- .../spark/graph/api/CypherSession.scala | 9 ++ .../spark/graph/api/GraphElementFrame.scala | 143 ++++++++++-------- .../graph/api/JavaPropertyGraphSuite.java | 10 +- .../spark/cypher/GraphExamplesSuite.scala | 24 ++- .../spark/cypher/PropertyGraphReadWrite.scala | 5 +- .../spark/cypher/PropertyGraphSuite.scala | 43 +++--- 6 files changed, 142 insertions(+), 92 deletions(-) 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 index 0aca517d85fba..43cb07579a793 100644 --- 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 @@ -254,4 +254,13 @@ trait CypherSession extends Logging { def buildNodeFrame(df: DataFrame): NodeFrameBuilder = new NodeFrameBuilder(df) + /** + * Returns a [[RelationshipFrameBuilder]] that can be used to construct a [[RelationshipFrame]]. + * + * @param df DataFrame containing a single relationship in each row + * @since 3.0.0 + */ + def buildRelationshipFrame(df: DataFrame): RelationshipFrameBuilder = + new RelationshipFrameBuilder(df) + } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 20bb1de30df5c..390e1c9f0a504 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -65,12 +65,12 @@ sealed abstract class GraphElementFrame { /** * Interface used to build a [[NodeFrame]]. * - * @param dataFrame DataFrame containing a single node in each row + * @param df DataFrame containing a single node in each row * @since 3.0.0 */ -final class NodeFrameBuilder(var dataFrame: DataFrame) { +final class NodeFrameBuilder(var df: DataFrame) { - private var idColumn: String = "id" + private var idColumn: String = CypherSession.ID_COLUMN private var labelSet: Set[String] = Set.empty private var properties: Map[String, String] = Map.empty @@ -119,7 +119,7 @@ final class NodeFrameBuilder(var dataFrame: DataFrame) { * @since 3.0.0 */ def build(): NodeFrame = { - NodeFrame(dataFrame, idColumn, labelSet, properties) + NodeFrame(df, idColumn, labelSet, properties) } } @@ -143,81 +143,98 @@ case class NodeFrame private[graph] ( properties: Map[String, String]) extends GraphElementFrame -object RelationshipFrame { +/** + * Interface used to build a [[RelationshipFrame]]. + * + * @param df DataFrame containing a single relationship in each row + * @since 3.0.0 + */ +final class RelationshipFrameBuilder(val df: DataFrame) { + + 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 /** - * Describes how to map a DataFrame to relationships. - * - * All columns apart from the given identifier columns are mapped to relationship properties. - * - * @param df DataFrame 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 idColumn column that contains the relationship identifier * @since 3.0.0 */ - def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String): RelationshipFrame = { - val properties = (df.columns.toSet - idColumn - sourceIdColumn - targetIdColumn) - .map(columnName => columnName -> columnName) - .toMap + def idColumn(idColumn: String): RelationshipFrameBuilder = { + if (idColumn.isEmpty) { + throw new IllegalArgumentException("idColumn must not be empty") + } + this.idColumn = idColumn; + this + } - create(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + /** + * @param sourceIdColumn column that contains the source node identifier of the relationship + * @since 3.0.0 + */ + def sourceIdColumn(sourceIdColumn: String): RelationshipFrameBuilder = { + if (sourceIdColumn.isEmpty) { + throw new IllegalArgumentException("sourceIdColumn must not be empty") + } + this.sourceIdColumn = sourceIdColumn; + this } /** - * Describes how to map a DataFrame to relationships. - * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns + * @param targetIdColumn column that contains the target node identifier of the relationship * @since 3.0.0 */ - def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: Map[String, String]): RelationshipFrame = { - RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + def targetIdColumn(targetIdColumn: String): RelationshipFrameBuilder = { + if (targetIdColumn.isEmpty) { + throw new IllegalArgumentException("targetIdColumn must not be empty") + } + this.targetIdColumn = targetIdColumn; + this } /** - * Describes how to map a DataFrame to relationships. - * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns - * @since 3.0.0 - */ - def create( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: java.util.Map[String, String]): RelationshipFrame = { - RelationshipFrame( - df, - idColumn, - sourceIdColumn, - targetIdColumn, - relationshipType, - properties.asScala.toMap) + * @since 3.0.0 + */ + def relationshipType(relationshipType: String): RelationshipFrameBuilder = { + 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]): RelationshipFrameBuilder = { + 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]): RelationshipFrameBuilder = { + this.properties = properties.asScala.toMap + this + } + + /** + * Creates a [[RelationshipFrame]] from the specified builder parameters. + * + * @since 3.0.0 + */ + def build(): RelationshipFrame = { + maybeRelationshipType match { + case Some(relationshipType) => + RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, 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 index 0690703c755c4..cddd0e972fe48 100644 --- 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 @@ -81,8 +81,14 @@ public void testCreateFromNodeFrame() { .build(); Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); - RelationshipFrame knowsRelFrame = RelationshipFrame - .create(knowsDf, "id", "source", "target", "KNOWS"); + RelationshipFrame knowsRelFrame = cypherSession.buildRelationshipFrame(knowsDf) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Collections.singletonMap("since", "since")) + .build(); + PropertyGraph graph = cypherSession.createGraph( new NodeFrame[]{personNodeFrame}, 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 index 58cb400bff91f..659acfae8d399 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -44,7 +44,12 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") + val relationshipFrame: RelationshipFrame = cypherSession.buildRelationshipFrame(relationshipData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .build() val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) val result: CypherResult = graph.cypher( """ @@ -72,8 +77,19 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") - val knowsRF: RelationshipFrame = RelationshipFrame.create(knowsDF, "id", "source", "target", "KNOWS") - val teachesRF: RelationshipFrame = RelationshipFrame.create(teachesDF, "id", "source", "target", "TEACHES") + val knowsRF: RelationshipFrame = cypherSession.buildRelationshipFrame(knowsDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() + val teachesRF: RelationshipFrame = cypherSession.buildRelationshipFrame(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") @@ -91,7 +107,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") val knowsRF: RelationshipFrame = RelationshipFrame(knowsDF, "id", "source", "target", "KNOWS", Map("since" -> "col_since")) - val teachesRF: RelationshipFrame = RelationshipFrame.create(teachesDF, "id", "source", "target", "TEACHES") + val teachesRF: RelationshipFrame = RelationshipFrame(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") 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 index 95011181e2298..4772109baa53b 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -57,7 +57,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef .properties(Map("name" -> "name")) .build() - private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") + private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, "id", "source", "target", "KNOWS", Map.empty) test("save and load a graph") { val graph = cypherSession.createGraph(Array(nodeDataFrame), Array(relationshipFrame)) @@ -80,8 +80,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) .toDF("id", "source", "target", "since") - val relationshipFrame = - RelationshipFrame.create(relationshipData, "id", "source", "target", "KNOWS") + val relationshipFrame = RelationshipFrame(relationshipData, "id", "source", "target", "KNOWS", Map.empty) val writeGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) 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 index d0c359e4ec3eb..8a55fa2ac23d7 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -59,12 +59,13 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) .toDF("id", "source", "target", "since") - val relationshipFrame = RelationshipFrame.create( - relationshipData, - "id", - "source", - "target", - "KNOWS") + val relationshipFrame = cypherSession.buildRelationshipFrame(relationshipData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() val graph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) @@ -107,18 +108,19 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((1L, 2L, 1L))) .toDF("id", "source", "target") - val knowsRF = RelationshipFrame.create( - knowsDF, - "id", - "source", - "target", - "KNOWS") - val teachesRF = RelationshipFrame.create( - teachesDF, - "id", - "source", - "target", - "TEACHES") + val knowsRF = cypherSession.buildRelationshipFrame(knowsDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() + val teachesRF = cypherSession.buildRelationshipFrame(teachesDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("TEACHES") + .build() val graph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) @@ -184,12 +186,13 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "target", relationshipType = "KNOWS", properties = Map("since" -> "col_since")) - val teachesRF = RelationshipFrame.create( + val teachesRF = RelationshipFrame( teachesDF, "id", "source", "target", - "TEACHES") + "TEACHES", + Map.empty) val graph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) From ccb5da47b69db91f78b1b6ab0192ef4ac4d60695 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 10 Oct 2019 09:59:33 +0200 Subject: [PATCH 101/123] Address review comments Co-authored-by: Mats Rydberg --- .../spark/graph/api/CypherSession.scala | 5 -- .../spark/graph/api/PropertyGraphWriter.scala | 4 +- .../graph/api/JavaPropertyGraphSuite.java | 28 ++++---- .../spark/cypher/PropertyGraphSuite.scala | 69 +++++++++---------- 4 files changed, 49 insertions(+), 57 deletions(-) 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 index 43cb07579a793..f3e14a2936713 100644 --- 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 @@ -64,7 +64,6 @@ trait CypherSession extends Logging { /** * Executes a Cypher query on the given input graph. * - * @see Cypher Manual * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @since 3.0.0 @@ -90,8 +89,6 @@ trait CypherSession extends Logging { * RETURN n * }}} * - * @see Cypher Manual - * @see Parameters * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query @@ -118,8 +115,6 @@ trait CypherSession extends Logging { * RETURN n * }}} * - * @see Cypher Manual - * @see Parameters * @param graph [[PropertyGraph]] on which the query is executed * @param query Cypher query to execute * @param parameters parameters used by the Cypher query 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 index 528cf5a737088..3b71f763636ca 100644 --- 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 @@ -64,9 +64,7 @@ abstract class PropertyGraphWriter(val graph: PropertyGraph) { saveMode.toLowerCase(Locale.ROOT) match { case "overwrite" => mode(SaveMode.Overwrite) case "ignore" => mode(SaveMode.Ignore) - case "error" | "errorifexists" => mode(SaveMode.ErrorIfExists) - case "default" => this - case "append" => mode(SaveMode.Append) + case "error" | "errorifexists" | "default" => mode(SaveMode.ErrorIfExists) case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " + "Accepted save modes are 'overwrite', 'ignore', 'error', 'errorifexists'.") } 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 index cddd0e972fe48..976c81b51f652 100644 --- 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 @@ -60,16 +60,16 @@ public void tearDown() { @Test public void testCreateFromNodeFrame() { StructType personSchema = createSchema( - Lists.newArrayList("id", "name"), - Lists.newArrayList(LongType, StringType)); + Lists.newArrayList("id", "name"), + Lists.newArrayList(LongType, StringType)); List personData = Arrays.asList( - RowFactory.create(0L, "Alice"), - RowFactory.create(1L, "Bob")); + RowFactory.create(0L, "Alice"), + RowFactory.create(1L, "Bob")); StructType knowsSchema = createSchema( - Lists.newArrayList("id", "source", "target", "since"), - Lists.newArrayList(LongType, LongType, LongType, IntegerType)); + Lists.newArrayList("id", "source", "target", "since"), + Lists.newArrayList(LongType, LongType, LongType, IntegerType)); List knowsData = Collections.singletonList(RowFactory.create(0L, 0L, 1L, 1984)); @@ -82,17 +82,17 @@ public void testCreateFromNodeFrame() { Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); RelationshipFrame knowsRelFrame = cypherSession.buildRelationshipFrame(knowsDf) - .idColumn("id") - .sourceIdColumn("source") - .targetIdColumn("target") - .relationshipType("KNOWS") - .properties(Collections.singletonMap("since", "since")) - .build(); + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Collections.singletonMap("since", "since")) + .build(); PropertyGraph graph = cypherSession.createGraph( - new NodeFrame[]{personNodeFrame}, - new RelationshipFrame[]{knowsRelFrame}); + new NodeFrame[]{personNodeFrame}, + new RelationshipFrame[]{knowsRelFrame}); List result = graph.nodes().collectAsList(); Assert.assertEquals(2, result.size()); } 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 index 8a55fa2ac23d7..e91de05407dd3 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -33,6 +33,40 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers def cypherSession: CypherSession = SparkCypherSession.create + lazy val nodes: DataFrame = 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: DataFrame = 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 NodeFrame") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") val nodeFrame = cypherSession.buildNodeFrame(nodeData) @@ -229,41 +263,6 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers checkAnswer(graph.relationships, expectedRelDf) } - lazy val nodes: DataFrame = 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: DataFrame = 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("select nodes via label set") { val graph = cypherSession.createGraph(nodes, relationships) val nodeFrame = graph.nodeFrame(Array("Person", "Teacher")) From 68c058a93eed5a82a816085e937e4ede0ef18ddb Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Thu, 10 Oct 2019 10:07:19 +0200 Subject: [PATCH 102/123] Extract ElementFrame classes and builders into separate files Co-authored-by: Mats Rydberg --- .../spark/graph/api/GraphElementFrame.scala | 205 +----------------- .../apache/spark/graph/api/NodeFrame.scala | 39 ++++ .../spark/graph/api/NodeFrameBuilder.scala | 84 +++++++ .../spark/graph/api/RelationshipFrame.scala | 46 ++++ .../graph/api/RelationshipFrameBuilder.scala | 116 ++++++++++ 5 files changed, 286 insertions(+), 204 deletions(-) create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala create mode 100644 graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 390e1c9f0a504..03160c7f8a436 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -17,8 +17,6 @@ package org.apache.spark.graph.api -import scala.collection.JavaConverters._ - import org.apache.spark.sql.DataFrame /** @@ -29,7 +27,7 @@ import org.apache.spark.sql.DataFrame * * @since 3.0.0 */ -sealed abstract class GraphElementFrame { +abstract class GraphElementFrame { /** * Initial DataFrame that can still contain unmapped, arbitrarily ordered columns. @@ -61,204 +59,3 @@ sealed abstract class GraphElementFrame { def properties: Map[String, String] } - -/** - * Interface used to build a [[NodeFrame]]. - * - * @param df DataFrame containing a single node in each row - * @since 3.0.0 - */ -final class NodeFrameBuilder(var df: DataFrame) { - - 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): NodeFrameBuilder = { - 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]): NodeFrameBuilder = { - this.labelSet = labelSet.toSet - this - } - - /** - * @param properties mapping from property keys to corresponding columns - * @since 3.0.0 - */ - def properties(properties: Map[String, String]): NodeFrameBuilder = { - 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]): NodeFrameBuilder = { - this.properties = properties.asScala.toMap - this - } - - /** - * Creates a `NodeFrame` from the specified builder parameters. - * - * @since 3.0.0 - */ - def build(): NodeFrame = { - NodeFrame(df, idColumn, labelSet, properties) - } - -} - -/** - * Describes how to map a DataFrame to nodes. - * - * Each row in the DataFrame represents a node which has exactly the labels defined by the given - * label set. - * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns - * @since 3.0.0 - */ -case class NodeFrame private[graph] ( - df: DataFrame, - idColumn: String, - labelSet: Set[String], - properties: Map[String, String]) - extends GraphElementFrame - -/** - * Interface used to build a [[RelationshipFrame]]. - * - * @param df DataFrame containing a single relationship in each row - * @since 3.0.0 - */ -final class RelationshipFrameBuilder(val df: DataFrame) { - - 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): RelationshipFrameBuilder = { - 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): RelationshipFrameBuilder = { - 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): RelationshipFrameBuilder = { - 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): RelationshipFrameBuilder = { - 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]): RelationshipFrameBuilder = { - 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]): RelationshipFrameBuilder = { - this.properties = properties.asScala.toMap - this - } - - /** - * Creates a [[RelationshipFrame]] from the specified builder parameters. - * - * @since 3.0.0 - */ - def build(): RelationshipFrame = { - maybeRelationshipType match { - case Some(relationshipType) => - RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) - case None => throw new IllegalArgumentException("Relationship type must be set.") - } - } -} - -/** - * Describes how to map a DataFrame to relationships. - * - * Each row in the DataFrame represents a relationship with the given relationship type. - * - * @param df DataFrame 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 properties mapping from property keys to corresponding columns - * @since 3.0.0 - */ -case class RelationshipFrame private[graph] ( - df: DataFrame, - idColumn: String, - sourceIdColumn: String, - targetIdColumn: String, - relationshipType: String, - properties: Map[String, String]) - extends GraphElementFrame { - - override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) - -} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala new file mode 100644 index 0000000000000..3ee21c974dcec --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.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.graph.api + +import org.apache.spark.sql.DataFrame + +/** + * Describes how to map a DataFrame to nodes. + * + * Each row in the DataFrame represents a node which has exactly the labels defined by the given + * label set. + * + * @param df DataFrame 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ +case class NodeFrame private[graph] ( + df: DataFrame, + idColumn: String, + labelSet: Set[String], + properties: Map[String, String]) + extends GraphElementFrame diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala new file mode 100644 index 0000000000000..9ec79ece4edd8 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala @@ -0,0 +1,84 @@ +/* + * 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.sql.DataFrame + +import scala.collection.JavaConverters._ + +/** + * Interface used to build a [[NodeFrame]]. + * + * @param df DataFrame containing a single node in each row + * @since 3.0.0 + */ +final class NodeFrameBuilder(var df: DataFrame) { + + 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): NodeFrameBuilder = { + 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]): NodeFrameBuilder = { + this.labelSet = labelSet.toSet + this + } + + /** + * @param properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ + def properties(properties: Map[String, String]): NodeFrameBuilder = { + 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]): NodeFrameBuilder = { + this.properties = properties.asScala.toMap + this + } + + /** + * Creates a `NodeFrame` from the specified builder parameters. + * + * @since 3.0.0 + */ + def build(): NodeFrame = { + NodeFrame(df, idColumn, labelSet, properties) + } + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala new file mode 100644 index 0000000000000..bd2efff4f3edd --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.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.sql.DataFrame + +/** + * Describes how to map a DataFrame to relationships. + * + * Each row in the DataFrame represents a relationship with the given relationship type. + * + * @param df DataFrame 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 properties mapping from property keys to corresponding columns + * @since 3.0.0 + */ +case class RelationshipFrame private[graph] ( + df: DataFrame, + idColumn: String, + sourceIdColumn: String, + targetIdColumn: String, + relationshipType: String, + properties: Map[String, String]) + extends GraphElementFrame { + + override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) + +} diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala new file mode 100644 index 0000000000000..f39e0641f2788 --- /dev/null +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala @@ -0,0 +1,116 @@ +/* + * 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.sql.DataFrame + +import scala.collection.JavaConverters._ + +/** + * Interface used to build a [[RelationshipFrame]]. + * + * @param df DataFrame containing a single relationship in each row + * @since 3.0.0 + */ +final class RelationshipFrameBuilder(val df: DataFrame) { + + 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): RelationshipFrameBuilder = { + 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): RelationshipFrameBuilder = { + 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): RelationshipFrameBuilder = { + 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): RelationshipFrameBuilder = { + 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]): RelationshipFrameBuilder = { + 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]): RelationshipFrameBuilder = { + this.properties = properties.asScala.toMap + this + } + + /** + * Creates a [[RelationshipFrame]] from the specified builder parameters. + * + * @since 3.0.0 + */ + def build(): RelationshipFrame = { + maybeRelationshipType match { + case Some(relationshipType) => + RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + case None => throw new IllegalArgumentException("Relationship type must be set.") + } + } +} From f7adb949bd1f636f3158ebf34479353c392d70c9 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 11 Oct 2019 11:24:38 +0200 Subject: [PATCH 103/123] Use Dataset[Row] over DataFrame Co-authored-by: Mats Rydberg --- .../apache/spark/graph/api/CypherResult.scala | 4 ++-- .../spark/graph/api/CypherSession.scala | 24 +++++++++---------- .../spark/graph/api/GraphElementFrame.scala | 4 ++-- .../apache/spark/graph/api/NodeFrame.scala | 6 ++--- .../spark/graph/api/NodeFrameBuilder.scala | 6 ++--- .../spark/graph/api/PropertyGraph.scala | 6 ++--- .../spark/graph/api/RelationshipFrame.scala | 6 ++--- .../graph/api/RelationshipFrameBuilder.scala | 10 ++++---- .../spark/cypher/PropertyGraphSuite.scala | 11 +++++---- 9 files changed, 39 insertions(+), 38 deletions(-) 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 index cee3042a5b87f..6372ca88b4ddc 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Dataset, Row} /** * Result of a Cypher query. @@ -40,5 +40,5 @@ trait CypherResult { * (e.g. `MATCH (n:Person) RETURN n` results in the columns `n`, `n:Person` and `n_name`). * @since 3.0.0 */ - def df: DataFrame + def df: 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 index f3e14a2936713..a7e8b35993c9e 100644 --- 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 @@ -20,7 +20,7 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.types.{BooleanType, StructType} /** @@ -143,7 +143,7 @@ trait CypherSession extends Logging { /** * Creates a [[PropertyGraph]] from nodes and relationships. * - * The given DataFrames need to adhere to the following column naming conventions: + * The given dataset needs to adhere to the following column naming conventions: * * {{{ * Id column: `$ID` (nodes and relationships) @@ -156,15 +156,15 @@ trait CypherSession extends Logging { * Property columns: `{Property_Key}` (nodes and relationships) * }}} * - * @note It is recommended to cache the input DataFrames if they represent multiple label sets and + * @note It is recommended to cache the input datasets if they represent multiple label sets and * relationship types. * * @see [[CypherSession]] - * @param nodes node DataFrame - * @param relationships relationship DataFrame + * @param nodes node dataset + * @param relationships relationship dataset * @since 3.0.0 */ - def createGraph(nodes: DataFrame, relationships: DataFrame): PropertyGraph = { + def createGraph(nodes: Dataset[Row], relationships: Dataset[Row]): PropertyGraph = { def validateLabelColumns(schema: StructType, columns: Set[String]): Unit = { schema.fields .filter(f => columns.contains(f.name)) @@ -243,19 +243,19 @@ trait CypherSession extends Logging { /** * Returns a [[NodeFrameBuilder]] that can be used to construct a [[NodeFrame]]. * - * @param df DataFrame containing a single node in each row + * @param ds Dataset containing a single node in each row * @since 3.0.0 */ - def buildNodeFrame(df: DataFrame): NodeFrameBuilder = - new NodeFrameBuilder(df) + def buildNodeFrame(ds: Dataset[Row]): NodeFrameBuilder = + new NodeFrameBuilder(ds) /** * Returns a [[RelationshipFrameBuilder]] that can be used to construct a [[RelationshipFrame]]. * - * @param df DataFrame containing a single relationship in each row + * @param ds Dataset containing a single relationship in each row * @since 3.0.0 */ - def buildRelationshipFrame(df: DataFrame): RelationshipFrameBuilder = - new RelationshipFrameBuilder(df) + def buildRelationshipFrame(ds: Dataset[Row]): RelationshipFrameBuilder = + new RelationshipFrameBuilder(ds) } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index 03160c7f8a436..e8c19f8b41b7b 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -17,7 +17,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Dataset, Row} /** * A [[PropertyGraph]] is created from GraphElementFrames. @@ -34,7 +34,7 @@ abstract class GraphElementFrame { * * @since 3.0.0 */ - def df: DataFrame + def df: Dataset[Row] /** * Name of the column that contains the graph element identifier. diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala index 3ee21c974dcec..163120c0d511c 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala @@ -17,7 +17,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Dataset, Row} /** * Describes how to map a DataFrame to nodes. @@ -32,8 +32,8 @@ import org.apache.spark.sql.DataFrame * @since 3.0.0 */ case class NodeFrame private[graph] ( - df: DataFrame, + df: Dataset[Row], idColumn: String, labelSet: Set[String], properties: Map[String, String]) - extends GraphElementFrame + extends GraphElementFrame diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala index 9ec79ece4edd8..53a5a6cd575aa 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala @@ -17,17 +17,17 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame - import scala.collection.JavaConverters._ +import org.apache.spark.sql.{Dataset, Row} + /** * Interface used to build a [[NodeFrame]]. * * @param df DataFrame containing a single node in each row * @since 3.0.0 */ -final class NodeFrameBuilder(var df: DataFrame) { +final class NodeFrameBuilder(var df: Dataset[Row]) { private var idColumn: String = CypherSession.ID_COLUMN private var labelSet: Set[String] = Set.empty 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 index 8d15b7e731755..18adb175d758b 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Dataset, Row} /** * A Property Graph as defined by the openCypher Property Graph Data Model. @@ -108,7 +108,7 @@ abstract class PropertyGraph { * @see `org.apache.spark.graph.api.CypherSession.createGraph(nodes, relationships)` * @since 3.0.0 */ - def nodes: DataFrame + def nodes: Dataset[Row] /** * Returns a DataFrame that contains a row for each relationship in this @@ -127,7 +127,7 @@ abstract class PropertyGraph { * @see `org.apache.spark.graph.api.CypherSession.createGraph(nodes, relationships)` * @since 3.0.0 */ - def relationships: DataFrame + def relationships: Dataset[Row] /** * Interface for saving this `PropertyGraph` out into external storage. diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala index bd2efff4f3edd..94a168a24fc5c 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala @@ -17,7 +17,7 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Dataset, Row} /** * Describes how to map a DataFrame to relationships. @@ -33,13 +33,13 @@ import org.apache.spark.sql.DataFrame * @since 3.0.0 */ case class RelationshipFrame private[graph] ( - df: DataFrame, + df: Dataset[Row], idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String, properties: Map[String, String]) - extends GraphElementFrame { + extends GraphElementFrame { override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala index f39e0641f2788..364bf807d89b1 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala @@ -17,17 +17,17 @@ package org.apache.spark.graph.api -import org.apache.spark.sql.DataFrame - import scala.collection.JavaConverters._ +import org.apache.spark.sql.{Dataset, Row} + /** * Interface used to build a [[RelationshipFrame]]. * * @param df DataFrame containing a single relationship in each row * @since 3.0.0 */ -final class RelationshipFrameBuilder(val df: DataFrame) { +final class RelationshipFrameBuilder(val df: Dataset[Row]) { private var idColumn: String = CypherSession.ID_COLUMN private var sourceIdColumn: String = CypherSession.SOURCE_ID_COLUMN @@ -108,8 +108,8 @@ final class RelationshipFrameBuilder(val df: DataFrame) { */ def build(): RelationshipFrame = { maybeRelationshipType match { - case Some(relationshipType) => - RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relationshipType, properties) + case Some(relType) => + RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relType, properties) case None => throw new IllegalArgumentException("Relationship type must be set.") } } 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 index e91de05407dd3..8f71f98ee86fe 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -18,12 +18,13 @@ package org.apache.spark.cypher -import org.apache.spark.graph.api.CypherSession._ +import org.scalatest.Matchers + import org.apache.spark.graph.api.{CypherSession, NodeFrame, RelationshipFrame} +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 -import org.apache.spark.sql.{DataFrame, QueryTest} -import org.scalatest.Matchers class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers { @@ -33,7 +34,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers def cypherSession: CypherSession = SparkCypherSession.create - lazy val nodes: DataFrame = spark + lazy val nodes: Dataset[Row] = spark .createDataFrame( Seq( (0L, true, true, false, false, Some(42), Some("Alice"), None, None), @@ -53,7 +54,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "subject", "title") - lazy val relationships: DataFrame = spark + lazy val relationships: Dataset[Row] = spark .createDataFrame( Seq( (0L, 0L, 1L, true, false), From 0c18e7ebdcb17326751f70cd30866d133a01d584 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 11 Oct 2019 11:26:52 +0200 Subject: [PATCH 104/123] Change indentation to 2 in JavaPropertyGraphSuite --- .../graph/api/JavaPropertyGraphSuite.java | 126 +++++++++--------- 1 file changed, 63 insertions(+), 63 deletions(-) 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 index 976c81b51f652..75b4f9d9127eb 100644 --- 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 @@ -40,68 +40,68 @@ import static org.apache.spark.sql.types.DataTypes.*; public abstract class JavaPropertyGraphSuite implements Serializable { - private transient TestSparkSession spark; - private transient CypherSession cypherSession; - - 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 testCreateFromNodeFrame() { - 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); - NodeFrame personNodeFrame = cypherSession.buildNodeFrame(personDf) - .idColumn("id") - .labelSet(new String[]{"Person"}) - .properties(Collections.singletonMap("name", "name")) - .build(); - - Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); - RelationshipFrame knowsRelFrame = cypherSession.buildRelationshipFrame(knowsDf) - .idColumn("id") - .sourceIdColumn("source") - .targetIdColumn("target") - .relationshipType("KNOWS") - .properties(Collections.singletonMap("since", "since")) - .build(); - - - PropertyGraph graph = cypherSession.createGraph( - new NodeFrame[]{personNodeFrame}, - new RelationshipFrame[]{knowsRelFrame}); - 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); + private transient TestSparkSession spark; + private transient CypherSession cypherSession; + + 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 testCreateFromNodeFrame() { + 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); + NodeFrame personNodeFrame = cypherSession.buildNodeFrame(personDf) + .idColumn("id") + .labelSet(new String[]{"Person"}) + .properties(Collections.singletonMap("name", "name")) + .build(); + + Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); + RelationshipFrame knowsRelFrame = cypherSession.buildRelationshipFrame(knowsDf) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Collections.singletonMap("since", "since")) + .build(); + + + PropertyGraph graph = cypherSession.createGraph( + new NodeFrame[]{personNodeFrame}, + new RelationshipFrame[]{knowsRelFrame}); + 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); + } } From 1792d6ca066fc634d36326d180c95983911a33a3 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Fri, 11 Oct 2019 14:10:43 +0200 Subject: [PATCH 105/123] Add clarifying comment on abstract methods in test suites --- .../org/apache/spark/graph/api/JavaPropertyGraphSuite.java | 6 ++++++ .../scala/org/apache/spark/cypher/PropertyGraphSuite.scala | 6 ++++++ 2 files changed, 12 insertions(+) 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 index 75b4f9d9127eb..5fd09213ae426 100644 --- 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 @@ -43,6 +43,12 @@ 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 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 index 8f71f98ee86fe..de76877cbdfd9 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -32,6 +32,12 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers 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 From 624a77a0187a62de149894866acb1c1c0c7ccfb7 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Sat, 12 Oct 2019 09:41:46 +0200 Subject: [PATCH 106/123] Refactor ElementFrame conversion methods into CypherSession object --- .../spark/graph/api/CypherSession.scala | 154 ++++++++++-------- 1 file changed, 86 insertions(+), 68 deletions(-) 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 index a7e8b35993c9e..865c9aafbf282 100644 --- 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 @@ -19,7 +19,8 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ -import org.apache.spark.internal.Logging +import org.slf4j.LoggerFactory + import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.types.{BooleanType, StructType} @@ -47,6 +48,86 @@ object CypherSession { * Naming convention both for node label and relationship type prefixes. */ val LABEL_COLUMN_PREFIX = ":" + + /** + * Extracts [[NodeFrame]]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 extractNodeFrames(nodes: Dataset[Row]): Set[NodeFrame] = { + val labelColumns = nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet + validateLabelColumns(nodes.schema, labelColumns) + + 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 frames.") + 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(_ && _) + + NodeFrame(nodes.filter(predicate), ID_COLUMN, labelSet.map(_.substring(1)), nodeProperties) + } + } + + /** + * Extracts [[RelationshipFrame]]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 extractRelationshipFrames(relationships: Dataset[Row]): Set[RelationshipFrame] = { + val relColumns = relationships.columns.toSet + val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) + validateLabelColumns(relationships.schema, relTypeColumns) + 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) + + RelationshipFrame( + relationships.filter(predicate), + ID_COLUMN, + SOURCE_ID_COLUMN, + TARGET_ID_COLUMN, + relTypeColumn.substring(1), + relProperties) + } + } + + private def validateLabelColumns(schema: StructType, columns: Set[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.") + } + }) + } + } /** @@ -57,7 +138,7 @@ object CypherSession { * * @since 3.0.0 */ -trait CypherSession extends Logging { +trait CypherSession { def sparkSession: SparkSession @@ -165,72 +246,9 @@ trait CypherSession extends Logging { * @since 3.0.0 */ def createGraph(nodes: Dataset[Row], relationships: Dataset[Row]): PropertyGraph = { - def validateLabelColumns(schema: StructType, columns: Set[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.") - } - }) - } - - val idColumn = CypherSession.ID_COLUMN - val sourceIdColumn = CypherSession.SOURCE_ID_COLUMN - val targetIdColumn = CypherSession.TARGET_ID_COLUMN - - val labelColumns = nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet - validateLabelColumns(nodes.schema, labelColumns) - - val nodeProperties = (nodes.columns.toSet - idColumn -- labelColumns) - .map(col => col -> col) - .toMap - - val labelCount = labelColumns.size - if (labelCount > 5) { - log.warn( - s"$labelCount label columns will result in ${Math.pow(labelCount, 2)} node frames.") - 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 - - val nodeFrames = labelSets.map { labelSet => - val predicate = labelColumns - .map { labelColumn => - if (labelSet.contains(labelColumn)) { - nodes.col(labelColumn) - } else { - !nodes.col(labelColumn) - } - } - .reduce(_ && _) - - NodeFrame(nodes.filter(predicate), idColumn, labelSet.map(_.substring(1)), nodeProperties) - } - - val relColumns = relationships.columns.toSet - val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) - validateLabelColumns(relationships.schema, relTypeColumns) - val propertyColumns = relColumns - idColumn - sourceIdColumn - targetIdColumn -- relTypeColumns - val relProperties = propertyColumns.map(col => col -> col).toMap - val relFrames = relTypeColumns.map { relTypeColumn => - val predicate = relationships.col(relTypeColumn) - - RelationshipFrame( - relationships.filter(predicate), - idColumn, - sourceIdColumn, - targetIdColumn, - relTypeColumn.substring(1), - relProperties) - } - - createGraph(nodeFrames.toArray, relFrames.toArray) + val nodeFrames = CypherSession.extractNodeFrames(nodes) + val relationshipFrames = CypherSession.extractRelationshipFrames(relationships) + createGraph(nodeFrames.toArray, relationshipFrames.toArray) } /** From 66150181fb14d5161ebcf46eda9d9544b32c2961 Mon Sep 17 00:00:00 2001 From: Mats Rydberg Date: Fri, 11 Oct 2019 17:54:39 +0200 Subject: [PATCH 107/123] Use Dataset[Row] over DataFrame - Rename variables - Rename doc comments --- .../apache/spark/graph/api/CypherResult.scala | 4 +- .../spark/graph/api/GraphElementFrame.scala | 6 +-- .../apache/spark/graph/api/NodeFrame.scala | 10 ++--- .../spark/graph/api/NodeFrameBuilder.scala | 6 +-- .../spark/graph/api/PropertyGraph.scala | 8 ++-- .../spark/graph/api/RelationshipFrame.scala | 8 ++-- .../graph/api/RelationshipFrameBuilder.scala | 6 +-- .../spark/cypher/SparkCypherFunctions.scala | 8 ++-- .../spark/cypher/SparkCypherRecords.scala | 4 +- .../spark/cypher/SparkCypherResult.scala | 4 +- .../spark/cypher/SparkCypherSession.scala | 6 +-- .../org/apache/spark/cypher/SparkTable.scala | 16 +++---- .../adapters/RelationalGraphAdapter.scala | 10 ++--- .../cypher/conversions/ExprConversions.scala | 10 ++--- .../GraphElementFrameConversions.scala | 26 +++++------ .../spark/cypher/io/ReadWriteGraph.scala | 6 +-- .../spark/cypher/GraphExamplesSuite.scala | 44 +++++++++---------- .../spark/cypher/PropertyGraphReadWrite.scala | 8 ++-- .../spark/cypher/PropertyGraphSuite.scala | 4 +- .../construction/ScanGraphFactory.scala | 4 +- 20 files changed, 99 insertions(+), 99 deletions(-) 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 index 6372ca88b4ddc..a3344e07655b3 100644 --- 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.{Dataset, Row} /** * Result of a Cypher query. * - * Wraps a DataFrame that contains the result rows. + * Wraps a Dataset that contains the result rows. * * @since 3.0.0 */ @@ -40,5 +40,5 @@ trait CypherResult { * (e.g. `MATCH (n:Person) RETURN n` results in the columns `n`, `n:Person` and `n_name`). * @since 3.0.0 */ - def df: Dataset[Row] + def ds: Dataset[Row] } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala index e8c19f8b41b7b..866dbf36d61a7 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala @@ -23,18 +23,18 @@ import org.apache.spark.sql.{Dataset, Row} * A [[PropertyGraph]] is created from GraphElementFrames. * * A graph element is either a node or a relationship. - * A GraphElementFrame wraps a DataFrame and describes how it maps to graph elements. + * A GraphElementFrame wraps a Dataset and describes how it maps to graph elements. * * @since 3.0.0 */ abstract class GraphElementFrame { /** - * Initial DataFrame that can still contain unmapped, arbitrarily ordered columns. + * Initial Dataset that can still contain unmapped, arbitrarily ordered columns. * * @since 3.0.0 */ - def df: Dataset[Row] + def ds: Dataset[Row] /** * Name of the column that contains the graph element identifier. diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala index 163120c0d511c..8c4a7eb61c75d 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala @@ -20,19 +20,19 @@ package org.apache.spark.graph.api import org.apache.spark.sql.{Dataset, Row} /** - * Describes how to map a DataFrame to nodes. + * Describes how to map a Dataset to nodes. * - * Each row in the DataFrame represents a node which has exactly the labels defined by the given + * Each row in the Dataset represents a node which has exactly the labels defined by the given * label set. * - * @param df DataFrame containing a single node in each row + * @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 properties mapping from property keys to corresponding columns * @since 3.0.0 */ -case class NodeFrame private[graph] ( - df: Dataset[Row], +case class NodeFrame private[graph]( + ds: Dataset[Row], idColumn: String, labelSet: Set[String], properties: Map[String, String]) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala index 53a5a6cd575aa..c4fdca9ac94ec 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.{Dataset, Row} /** * Interface used to build a [[NodeFrame]]. * - * @param df DataFrame containing a single node in each row + * @param ds Dataset containing a single node in each row * @since 3.0.0 */ -final class NodeFrameBuilder(var df: Dataset[Row]) { +final class NodeFrameBuilder(var ds: Dataset[Row]) { private var idColumn: String = CypherSession.ID_COLUMN private var labelSet: Set[String] = Set.empty @@ -78,7 +78,7 @@ final class NodeFrameBuilder(var df: Dataset[Row]) { * @since 3.0.0 */ def build(): NodeFrame = { - NodeFrame(df, idColumn, labelSet, properties) + NodeFrame(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 index 18adb175d758b..3c922653ff810 100644 --- 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 @@ -95,9 +95,9 @@ abstract class PropertyGraph { def relationshipFrame(relationshipType: String): RelationshipFrame /** - * Returns a DataFrame that contains a row for each node in this graph. + * Returns a Dataset that contains a row for each node in this graph. * - * The DataFrame adheres to the following column naming conventions: + * The Dataset adheres to the following column naming conventions: * * {{{ * Id column: `$ID` @@ -111,10 +111,10 @@ abstract class PropertyGraph { def nodes: Dataset[Row] /** - * Returns a DataFrame that contains a row for each relationship in this + * Returns a Dataset that contains a row for each relationship in this * graph. * - * The DataFrame adheres to column naming conventions: + * The Dataset adheres to column naming conventions: * * {{{ * Id column: `$ID` diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala index 94a168a24fc5c..6046d013d3ec6 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala @@ -20,11 +20,11 @@ package org.apache.spark.graph.api import org.apache.spark.sql.{Dataset, Row} /** - * Describes how to map a DataFrame to relationships. + * Describes how to map a Dataset to relationships. * - * Each row in the DataFrame represents a relationship with the given relationship type. + * Each row in the Dataset represents a relationship with the given relationship type. * - * @param df DataFrame containing a single relationship in each row + * @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 @@ -33,7 +33,7 @@ import org.apache.spark.sql.{Dataset, Row} * @since 3.0.0 */ case class RelationshipFrame private[graph] ( - df: Dataset[Row], + ds: Dataset[Row], idColumn: String, sourceIdColumn: String, targetIdColumn: String, diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala index 364bf807d89b1..ef5ad3d6253d7 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala @@ -24,10 +24,10 @@ import org.apache.spark.sql.{Dataset, Row} /** * Interface used to build a [[RelationshipFrame]]. * - * @param df DataFrame containing a single relationship in each row + * @param ds Dataset containing a single relationship in each row * @since 3.0.0 */ -final class RelationshipFrameBuilder(val df: Dataset[Row]) { +final class RelationshipFrameBuilder(val ds: Dataset[Row]) { private var idColumn: String = CypherSession.ID_COLUMN private var sourceIdColumn: String = CypherSession.SOURCE_ID_COLUMN @@ -109,7 +109,7 @@ final class RelationshipFrameBuilder(val df: Dataset[Row]) { def build(): RelationshipFrame = { maybeRelationshipType match { case Some(relType) => - RelationshipFrame(df, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + RelationshipFrame(ds, idColumn, sourceIdColumn, targetIdColumn, relType, properties) case None => throw new IllegalArgumentException("Relationship type must be set.") } } 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 index d9bf1b20e23e8..cb7bc0aba3ed2 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherFunctions.scala @@ -23,7 +23,7 @@ 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, DataFrame, functions} +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 @@ -119,13 +119,13 @@ object SparkCypherFunctions { new Column(StringTranslate(src.expr, matchingString.expr, replaceString.expr)) } - def column_for(expr: Expr)(implicit header: RecordHeader, df: DataFrame): Column = { + 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 (df.columns.contains(columnName)) { - df.col(columnName) + if (ds.columns.contains(columnName)) { + ds.col(columnName) } else { NULL_LIT } 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 index d514cf15aafcb..981931e593c1b 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherRecords.scala @@ -74,10 +74,10 @@ case class SparkCypherRecords( )(implicit session: SparkCypherSession) extends RelationalCypherRecords[DataFrameTable] with RecordBehaviour { override type Records = SparkCypherRecords - def df: DataFrame = table.df + def ds: Dataset[Row] = table.df override def cache(): SparkCypherRecords = { - df.cache() + ds.cache() this } 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 index e69c4ff486cd0..0f8e940be9f0b 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherResult.scala @@ -20,9 +20,9 @@ package org.apache.spark.cypher import org.apache.spark.cypher.SparkTable.DataFrameTable import org.apache.spark.graph.api.CypherResult -import org.apache.spark.sql.DataFrame +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 df: DataFrame = relationalTable.table.df + 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 index 2a3579502981d..970b6f41fea98 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -51,7 +51,7 @@ object SparkCypherSession { * 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.DataFrame]]s. + * 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 { @@ -78,8 +78,8 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) require(relationships.groupBy(_.relationshipType).forall(_._2.length == 1), "There can be at most one RelationshipFrame per relationship type") - val normalizedNodes = nodes.map(nf => nf.copy(df = normalizeDf(nf))) - val normalizedRelationships = relationships.map(rf => rf.copy(df = normalizeDf(rf))) + val normalizedNodes = nodes.map(nf => nf.copy(ds = normalizeDf(nf))) + val normalizedRelationships = relationships.map(rf => rf.copy(ds = normalizeDf(rf))) RelationalGraphAdapter(this, normalizedNodes, normalizedRelationships) } 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 index fd3dcf7164325..4698572e00b69 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkTable.scala @@ -21,7 +21,7 @@ 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.{Column, DataFrame, RelationalGroupedDataset, functions} +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 @@ -36,7 +36,7 @@ import scala.collection.JavaConverters._ object SparkTable { - implicit class DataFrameTable(val df: DataFrame) extends Table[DataFrameTable] { + implicit class DataFrameTable(val df: Dataset[Row]) extends Table[DataFrameTable] { private case class EmptyRow() @@ -123,7 +123,7 @@ object SparkTable { def withInnerExpr(expr: Expr)(f: Column => Column) = f(expr.asSparkSQLExpr(header, df, parameters)) - val data: Either[RelationalGroupedDataset, DataFrame] = + val data: Either[RelationalGroupedDataset, Dataset[Row]] = if (by.nonEmpty) { val columns = by.flatMap { expr => val withChildren = header.ownedBy(expr) @@ -206,9 +206,9 @@ object SparkTable { def unpersist(blocking: Boolean): DataFrameTable = df.unpersist(blocking) } - implicit class DataFrameOps(val df: DataFrame) extends AnyVal { + implicit class DataFrameOps(val df: Dataset[Row]) extends AnyVal { - def safeJoin(other: DataFrame, joinCols: Seq[(String, String)], joinType: String): DataFrame = { + 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))) @@ -222,7 +222,7 @@ object SparkTable { df.join(other, joinExpr, joinType) } - def safeDropColumns(names: String*): DataFrame = { + 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.") @@ -243,11 +243,11 @@ object SparkTable { df.schema.fields(df.schema.fieldIndex(columnName)) } - def safeRenameColumns(renames: (String, String)*): DataFrame = { + def safeRenameColumns(renames: (String, String)*): Dataset[Row] = { safeRenameColumns(renames.toMap) } - def safeRenameColumns(renames: Map[String, String]): DataFrame = { + def safeRenameColumns(renames: Map[String, String]): Dataset[Row] = { if (renames.isEmpty || renames.forall { case (oldColumn, newColumn) => oldColumn == newColumn }) { df } else { 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 index 95f7ebca87c42..2027efe9844f2 100644 --- 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 @@ -23,7 +23,7 @@ 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.{NodeFrame, PropertyGraph, PropertyGraphType, RelationshipFrame} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{Dataset, Row} import org.opencypher.okapi.api.types.{CTNode, CTRelationship} import org.opencypher.okapi.ir.api.expr.Var @@ -38,8 +38,8 @@ case class RelationalGraphAdapter( if (nodeFrames.isEmpty) { cypherSession.graphs.empty } else { - val nodeTables = nodeFrames.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.df) } - val relTables = relationshipFrames.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.df) } + val nodeTables = nodeFrames.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.ds) } + val relTables = relationshipFrames.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.ds) } cypherSession.graphs.create(nodeTables.head, nodeTables.tail ++ relTables: _*) } } @@ -48,7 +48,7 @@ case class RelationalGraphAdapter( private lazy val _relationshipFrame: Map[String, RelationshipFrame] = relationshipFrames.map(rf => rf.relationshipType -> rf).toMap - override def nodes: DataFrame = { + override def nodes: Dataset[Row] = { // TODO: move to API as default implementation val nodeVar = Var("n")(CTNode) val nodes = graph.nodes(nodeVar.name) @@ -65,7 +65,7 @@ case class RelationalGraphAdapter( df.select(selectColumns: _*) } - override def relationships: DataFrame = { + override def relationships: Dataset[Row] = { // TODO: move to API as default implementation val relVar = Var("r")(CTRelationship) val rels = graph.relationships(relVar.name) 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 index d5346c0a3531d..6a772bb0f0245 100644 --- 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 @@ -25,7 +25,7 @@ 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, DataFrame} +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._ @@ -47,7 +47,7 @@ object ExprConversions { * values from the evaluated children. */ def nullSafeConversion(expr: Expr)(withConvertedChildren: Seq[Column] => Column) - (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { + (implicit header: RecordHeader, ds: Dataset[Row], parameters: CypherMap): Column = { if (expr.cypherType == CTNull) { NULL_LIT } else if (expr.cypherType == CTTrue) { @@ -70,11 +70,11 @@ object ExprConversions { * 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 df the dataframe containing the data over 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, df: DataFrame, parameters: CypherMap): Column = { + 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) @@ -88,7 +88,7 @@ object ExprConversions { } private def convert(convertedChildren: Seq[Column]) - (implicit header: RecordHeader, df: DataFrame, parameters: CypherMap): Column = { + (implicit header: RecordHeader, ds: Dataset[Row], parameters: CypherMap): Column = { def child0: Column = convertedChildren.head 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 index aae7a5a85765c..bbe131bf4c565 100644 --- 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 @@ -19,38 +19,38 @@ package org.apache.spark.cypher.conversions import org.apache.spark.graph.api.GraphElementFrame -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.types.{BinaryType, ByteType, IntegerType, LongType, ShortType, StringType} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{Dataset, Row} object GraphElementFrameConversions { - def normalizeDf(frame: GraphElementFrame): DataFrame = { + def normalizeDf(frame: GraphElementFrame): Dataset[Row] = { val mappedColumnNames = frame.idColumns ++ frame.properties.values.toSeq.sorted - val mappedDf = if (mappedColumnNames == frame.df.columns.toSeq) { - frame.df + val mappedDf = if (mappedColumnNames == frame.ds.columns.toSeq) { + frame.ds } else { - frame.df.select(mappedColumnNames.map(frame.df.col): _*) + frame.ds.select(mappedColumnNames.map(frame.ds.col): _*) } - if (frame.idColumns.forall(idColumn => frame.df.schema(idColumn).dataType == BinaryType)) { + if (frame.idColumns.forall(idColumn => frame.ds.schema(idColumn).dataType == BinaryType)) { mappedDf } else { encodeIdColumns(mappedDf, frame.idColumns: _*) } } - private def encodeIdColumns(df: DataFrame, idColumnNames: String*): DataFrame = { + private def encodeIdColumns(ds: Dataset[Row], idColumnNames: String*): Dataset[Row] = { val encodedIdCols = idColumnNames.map { idColumnName => - val col = df.col(idColumnName) - df.schema(idColumnName).dataType match { + 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 = df.columns.filterNot(idColumnNames.contains) - val remainingCols = remainingColumnNames.map(df.col) - df.select(encodedIdCols ++ remainingCols: _*) + 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/io/ReadWriteGraph.scala b/graph/cypher/src/main/scala/org/apache/spark/cypher/io/ReadWriteGraph.scala index 06f55a4484839..22def6e1ef4d4 100644 --- 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 @@ -26,7 +26,7 @@ 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.{NodeFrame, RelationshipFrame} -import org.apache.spark.sql.{DataFrame, SparkSession} +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} @@ -83,7 +83,7 @@ object ReadWriteGraph { implicit class GraphExport(graph: RelationalCypherGraph[DataFrameTable]) { - def canonicalNodeTable(labels: Set[String]): DataFrame = { + 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) @@ -100,7 +100,7 @@ object ReadWriteGraph { nodeRecords.table.df.select(selectColumns: _*) } - def canonicalRelationshipTable(relType: String): DataFrame = { + def canonicalRelationshipTable(relType: String): Dataset[Row] = { val ct = CTRelationship(relType) val v = Var("r")(ct) val relRecords = graph.relationships(v.name, ct) 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 index 659acfae8d399..60de6f1a89a04 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.cypher import org.apache.spark.SparkFunSuite import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} -import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.{Dataset, Row, SaveMode} class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { test("create PropertyGraph from single NodeFrame") { - val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") + val nodeData: Dataset[Row] = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") val nodeFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) .idColumn("id") .labelSet(Array("Person")) @@ -33,12 +33,12 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { .build() val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) val result: CypherResult = graph.cypher("MATCH (n) RETURN n") - result.df.show() + result.ds.show() } test("create PropertyGraph from Node- and RelationshipFrames") { - val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") - val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))).toDF("id", "source", "target") + 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 nodeFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) .idColumn("id") .labelSet(Array("Person")) @@ -55,12 +55,12 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { """ |MATCH (a:Person)-[r:KNOWS]->(:Person) |RETURN a, r""".stripMargin) - result.df.show() + result.ds.show() } test("create PropertyGraph with multiple node and relationship types") { - val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") - val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") + val studentDF: Dataset[Row] = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") + val teacherDF: Dataset[Row] = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") val studentNF = cypherSession.buildNodeFrame(studentDF) .idColumn("id") @@ -74,8 +74,8 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { .properties(Map("name" -> "name", "subject" -> "subject")) .build() - val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "since") - val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") + 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: RelationshipFrame = cypherSession.buildRelationshipFrame(knowsDF) .idColumn("id") @@ -93,42 +93,42 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { 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.df.show() + result.ds.show() } test("create PropertyGraph with multiple node and relationship types and explicit property-to-column mappings") { - val studentDF: DataFrame = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "col_name", "col_age") - val teacherDF: DataFrame = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "col_name", "col_subject") + 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: NodeFrame = NodeFrame(studentDF, "id", Set("Person", "Student"), Map("name" -> "col_name", "age" -> "col_age")) val teacherNF: NodeFrame = NodeFrame(teacherDF, "id", Set("Person", "Teacher"), Map("name" -> "col_name", "subject" -> "col_subject")) - val knowsDF: DataFrame = spark.createDataFrame(Seq((0, 0, 1, 1984))).toDF("id", "source", "target", "col_since") - val teachesDF: DataFrame = spark.createDataFrame(Seq((1, 2, 1))).toDF("id", "source", "target") + 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: RelationshipFrame = RelationshipFrame(knowsDF, "id", "source", "target", "KNOWS", Map("since" -> "col_since")) val teachesRF: RelationshipFrame = RelationshipFrame(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.df.show() + result.ds.show() } test("create PropertyGraph with multiple node and relationship types stored in wide tables") { - val nodeDF: DataFrame = spark.createDataFrame(Seq( + 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: DataFrame = spark.createDataFrame(Seq( + 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.df.show() + result.ds.show() } test("save and load PropertyGraph") { @@ -152,10 +152,10 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { cypherSession .createGraph(nodesWithRetainedId, relsWithRetainedId) - .cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").df.show() + .cypher("MATCH (n:Student)-[:STUDY_AT]->(u:University) RETURN n, u").ds.show() } - lazy val nodes: DataFrame = spark.createDataFrame(Seq( + 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), @@ -164,7 +164,7 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { (5L, false, true, None, None, Some("Stanford")) )).toDF("$ID", ":Student", ":University", "name", "age", "title") - lazy val relationships: DataFrame = spark.createDataFrame(Seq( + lazy val relationships: Dataset[Row] = spark.createDataFrame(Seq( (0L, 0L, 1L, true, false), (1L, 0L, 3L, true, false), (2L, 1L, 3L, true, false), 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 index 4772109baa53b..b3eabfcce829e 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -21,7 +21,7 @@ package org.apache.spark.cypher import java.nio.file.Paths import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} -import org.apache.spark.sql.{DataFrame, QueryTest, SaveMode} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.junit.rules.TemporaryFolder import org.scalatest.BeforeAndAfterEach @@ -42,12 +42,12 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef private def basePath: String = s"file://${Paths.get(tempDir.getRoot.getAbsolutePath)}" - private lazy val nodeData: DataFrame = spark.createDataFrame(Seq( + private lazy val nodeData: Dataset[Row] = spark.createDataFrame(Seq( 0 -> "Alice", 1 -> "Bob" )).toDF("id", "name") - private lazy val relationshipData: DataFrame = spark.createDataFrame(Seq( + private lazy val relationshipData: Dataset[Row] = spark.createDataFrame(Seq( Tuple3(0, 0, 1) )).toDF("id", "source", "target") @@ -66,7 +66,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef val readGraph = cypherSession.read.load(basePath) readGraph.cypher( "MATCH (a:Person)-[:KNOWS]->(b:Person) RETURN a.name AS person1, b.name AS person2" - ).df.show() + ).ds.show() } test("save and loads a property graph") { 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 index de76877cbdfd9..7a2f4e2fabf1f 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -282,7 +282,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((convertId(2L), Some(22), Some("Carol"), Some("CS"), None: Option[String]))) .toDF(ID_COLUMN, "age", "name", "subject", "title") - checkAnswer(nodeFrame.df, expectedNodeDf) + checkAnswer(nodeFrame.ds, expectedNodeDf) } test("select relationships via type") { @@ -305,7 +305,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers (convertId(4L), convertId(3L), convertId(1L)))) .toDF(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN) - checkAnswer(relationshipFrame.df, expectedRelDf) + checkAnswer(relationshipFrame.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/construction/ScanGraphFactory.scala b/graph/cypher/src/test/scala/org/apache/spark/cypher/construction/ScanGraphFactory.scala index 566684c0f9ad4..9b375f89abb62 100644 --- 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 @@ -25,7 +25,7 @@ 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.{DataFrame, Row} +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 @@ -39,7 +39,7 @@ import scala.collection.JavaConverters._ object ScanGraphFactory extends CypherTestGraphFactory[SparkCypherSession] { - def encodeIdColumns(df: DataFrame, mapping: ElementMapping): DataFrame = { + 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 { From fa1c76e6b4437a7769a7f74a312c46d4412f9838 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 13 Oct 2019 10:43:26 -0700 Subject: [PATCH 108/123] Renamings --- .../apache/spark/graph/api/CypherResult.scala | 2 + .../spark/graph/api/CypherSession.scala | 50 +++++----- ...tFrame.scala => GraphElementDataset.scala} | 10 +- .../{NodeFrame.scala => NodeDataset.scala} | 6 +- ...Builder.scala => NodeDatasetBuilder.scala} | 20 ++-- .../spark/graph/api/PropertyGraph.scala | 18 ++-- .../spark/graph/api/PropertyGraphReader.scala | 3 + .../spark/graph/api/PropertyGraphType.scala | 3 + .../spark/graph/api/PropertyGraphWriter.scala | 2 + ...pFrame.scala => RelationshipDataset.scala} | 6 +- ...scala => RelationshipDatasetBuilder.scala} | 24 ++--- .../graph/api/JavaPropertyGraphSuite.java | 10 +- .../spark/cypher/SparkCypherSession.scala | 6 +- .../cypher/adapters/MappingAdapter.scala | 6 +- .../adapters/RelationalGraphAdapter.scala | 18 ++-- .../GraphElementFrameConversions.scala | 4 +- .../spark/cypher/io/ReadWriteGraph.scala | 10 +- .../io/SparkCypherPropertyGraphReader.scala | 2 +- .../spark/cypher/GraphExamplesSuite.scala | 36 +++---- .../spark/cypher/PropertyGraphReadWrite.scala | 10 +- .../spark/cypher/PropertyGraphSuite.scala | 94 ++++++++++--------- 21 files changed, 187 insertions(+), 153 deletions(-) rename graph/api/src/main/scala/org/apache/spark/graph/api/{GraphElementFrame.scala => GraphElementDataset.scala} (85%) rename graph/api/src/main/scala/org/apache/spark/graph/api/{NodeFrame.scala => NodeDataset.scala} (92%) rename graph/api/src/main/scala/org/apache/spark/graph/api/{NodeFrameBuilder.scala => NodeDatasetBuilder.scala} (79%) rename graph/api/src/main/scala/org/apache/spark/graph/api/{RelationshipFrame.scala => RelationshipDataset.scala} (93%) rename graph/api/src/main/scala/org/apache/spark/graph/api/{RelationshipFrameBuilder.scala => RelationshipDatasetBuilder.scala} (79%) 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 index a3344e07655b3..a2fc4d231441e 100644 --- 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 @@ -17,6 +17,7 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** @@ -26,6 +27,7 @@ import org.apache.spark.sql.{Dataset, Row} * * @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). 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 index 865c9aafbf282..8de5baf7e91fc 100644 --- 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 @@ -21,12 +21,14 @@ 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 { /** @@ -50,15 +52,15 @@ object CypherSession { val LABEL_COLUMN_PREFIX = ":" /** - * Extracts [[NodeFrame]]s from a [[Dataset]] using column name conventions. + * 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 extractNodeFrames(nodes: Dataset[Row]): Set[NodeFrame] = { - val labelColumns = nodes.columns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)).toSet + def extractNodeDataset(nodes: Dataset[Row]): Set[NodeDataset] = { + val labelColumns = nodes.columns.filter(_.startsWith(LABEL_COLUMN_PREFIX)).toSet validateLabelColumns(nodes.schema, labelColumns) val nodeProperties = (nodes.columns.toSet - ID_COLUMN -- labelColumns) @@ -88,19 +90,19 @@ object CypherSession { } .reduce(_ && _) - NodeFrame(nodes.filter(predicate), ID_COLUMN, labelSet.map(_.substring(1)), nodeProperties) + NodeDataset(nodes.filter(predicate), ID_COLUMN, labelSet.map(_.substring(1)), nodeProperties) } } /** - * Extracts [[RelationshipFrame]]s from a [[Dataset]] using column name conventions. + * 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 extractRelationshipFrames(relationships: Dataset[Row]): Set[RelationshipFrame] = { + def extractRelationshipDataset(relationships: Dataset[Row]): Set[RelationshipDataset] = { val relColumns = relationships.columns.toSet val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) validateLabelColumns(relationships.schema, relTypeColumns) @@ -110,7 +112,7 @@ object CypherSession { relTypeColumns.map { relTypeColumn => val predicate = relationships.col(relTypeColumn) - RelationshipFrame( + RelationshipDataset( relationships.filter(predicate), ID_COLUMN, SOURCE_ID_COLUMN, @@ -138,6 +140,7 @@ object CypherSession { * * @since 3.0.0 */ +@Evolving trait CypherSession { def sparkSession: SparkSession @@ -209,17 +212,19 @@ trait CypherSession { } /** - * Creates a [[PropertyGraph]] from a sequence of [[NodeFrame]]s and [[RelationshipFrame]]s. - * At least one [[NodeFrame]] has to be provided. + * 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 [[NodeFrame]] and at most one - * [[RelationshipFrame]], respectively. + * For each label set and relationship type there can be at most one [[NodeDataset]] and at most + * one [[RelationshipDataset]], respectively. * - * @param nodes NodeFrames that define the nodes in the graph - * @param relationships RelationshipFrames that define the relationships in the graph + * @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[NodeFrame], relationships: Array[RelationshipFrame]): PropertyGraph + def createGraph( + nodes: Array[NodeDataset], + relationships: Array[RelationshipDataset]): PropertyGraph /** * Creates a [[PropertyGraph]] from nodes and relationships. @@ -246,8 +251,8 @@ trait CypherSession { * @since 3.0.0 */ def createGraph(nodes: Dataset[Row], relationships: Dataset[Row]): PropertyGraph = { - val nodeFrames = CypherSession.extractNodeFrames(nodes) - val relationshipFrames = CypherSession.extractRelationshipFrames(relationships) + val nodeFrames = CypherSession.extractNodeDataset(nodes) + val relationshipFrames = CypherSession.extractRelationshipDataset(relationships) createGraph(nodeFrames.toArray, relationshipFrames.toArray) } @@ -259,21 +264,22 @@ trait CypherSession { def read: PropertyGraphReader /** - * Returns a [[NodeFrameBuilder]] that can be used to construct a [[NodeFrame]]. + * 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 buildNodeFrame(ds: Dataset[Row]): NodeFrameBuilder = - new NodeFrameBuilder(ds) + def buildNodeDataset(ds: Dataset[Row]): NodeDatasetBuilder = + new NodeDatasetBuilder(ds) /** - * Returns a [[RelationshipFrameBuilder]] that can be used to construct a [[RelationshipFrame]]. + * 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 buildRelationshipFrame(ds: Dataset[Row]): RelationshipFrameBuilder = - new RelationshipFrameBuilder(ds) + def buildRelationshipDataset(ds: Dataset[Row]): RelationshipDatasetBuilder = + new RelationshipDatasetBuilder(ds) } diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataset.scala similarity index 85% rename from graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataset.scala index 866dbf36d61a7..5f84b8fbc0bc9 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/GraphElementDataset.scala @@ -17,17 +17,17 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** - * A [[PropertyGraph]] is created from GraphElementFrames. - * - * A graph element is either a node or a relationship. - * A GraphElementFrame wraps a Dataset and describes how it maps to graph elements. + * A [[PropertyGraph]] whose graph element is either a node or a relationship. + * A GraphElementDataset wraps a Dataset and describes how it maps to graph elements. * * @since 3.0.0 */ -abstract class GraphElementFrame { +@Evolving +abstract class GraphElementDataset { /** * Initial Dataset that can still contain unmapped, arbitrarily ordered columns. diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDataset.scala similarity index 92% rename from graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/NodeDataset.scala index 8c4a7eb61c75d..b87504d356de2 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDataset.scala @@ -17,6 +17,7 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** @@ -31,9 +32,10 @@ import org.apache.spark.sql.{Dataset, Row} * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ -case class NodeFrame private[graph]( +@Evolving +case class NodeDataset private[graph]( ds: Dataset[Row], idColumn: String, labelSet: Set[String], properties: Map[String, String]) - extends GraphElementFrame + extends GraphElementDataset diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDatasetBuilder.scala similarity index 79% rename from graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/NodeDatasetBuilder.scala index c4fdca9ac94ec..543fe5ffb4974 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/NodeFrameBuilder.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/NodeDatasetBuilder.scala @@ -19,15 +19,17 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** - * Interface used to build a [[NodeFrame]]. + * A builder for [[NodeDataset]]. * * @param ds Dataset containing a single node in each row * @since 3.0.0 */ -final class NodeFrameBuilder(var ds: Dataset[Row]) { +@Evolving +final class NodeDatasetBuilder(val ds: Dataset[Row]) { private var idColumn: String = CypherSession.ID_COLUMN private var labelSet: Set[String] = Set.empty @@ -37,7 +39,7 @@ final class NodeFrameBuilder(var ds: Dataset[Row]) { * @param idColumn column that contains the node identifier * @since 3.0.0 */ - def idColumn(idColumn: String): NodeFrameBuilder = { + def idColumn(idColumn: String): NodeDatasetBuilder = { if (idColumn.isEmpty) { throw new IllegalArgumentException("idColumn must not be empty") } @@ -49,7 +51,7 @@ final class NodeFrameBuilder(var ds: Dataset[Row]) { * @param labelSet labels that are assigned to all nodes * @since 3.0.0 */ - def labelSet(labelSet: Array[String]): NodeFrameBuilder = { + def labelSet(labelSet: Array[String]): NodeDatasetBuilder = { this.labelSet = labelSet.toSet this } @@ -58,7 +60,7 @@ final class NodeFrameBuilder(var ds: Dataset[Row]) { * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def properties(properties: Map[String, String]): NodeFrameBuilder = { + def properties(properties: Map[String, String]): NodeDatasetBuilder = { this.properties = properties this } @@ -67,18 +69,18 @@ final class NodeFrameBuilder(var ds: Dataset[Row]) { * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def properties(properties: java.util.Map[String, String]): NodeFrameBuilder = { + def properties(properties: java.util.Map[String, String]): NodeDatasetBuilder = { this.properties = properties.asScala.toMap this } /** - * Creates a `NodeFrame` from the specified builder parameters. + * Creates a `NodeDataset` from the specified builder parameters. * * @since 3.0.0 */ - def build(): NodeFrame = { - NodeFrame(ds, idColumn, labelSet, properties) + 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 index 3c922653ff810..9f7f179ee1ef5 100644 --- 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 @@ -17,6 +17,7 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** @@ -29,6 +30,7 @@ import org.apache.spark.sql.{Dataset, Row} * @see Property Graph Model * @since 3.0.0 */ +@Evolving abstract class PropertyGraph { /** @@ -77,22 +79,22 @@ abstract class PropertyGraph { cypherSession.cypher(this, query, parameters) /** - * Returns the [[NodeFrame]] for a given node label set. + * Returns the [[NodeDataset]] for a given node label set. * - * @param labelSet Label set used for NodeFrame lookup - * @return NodeFrame for the given label set + * @param labelSet Label set used for NodeDataset lookup + * @return NodeDataset for the given label set * @since 3.0.0 */ - def nodeFrame(labelSet: Array[String]): NodeFrame + def nodeDataset(labelSet: Array[String]): NodeDataset /** - * Returns the [[RelationshipFrame]] for a given relationship type. + * Returns the [[RelationshipDataset]] for a given relationship type. * - * @param relationshipType Relationship type used for RelationshipFrame lookup - * @return RelationshipFrame for the given relationship type + * @param relationshipType Relationship type used for RelationshipDataset lookup + * @return RelationshipDataset for the given relationship type * @since 3.0.0 */ - def relationshipFrame(relationshipType: String): RelationshipFrame + def relationshipDataset(relationshipType: String): RelationshipDataset /** * Returns a Dataset that contains a row for each node in this graph. 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 index 760f8fc9d769e..258a8238d45d7 100644 --- 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 @@ -17,6 +17,9 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving + +@Evolving abstract class PropertyGraphReader(session: CypherSession) { protected var format: String = diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala index 1aabd23a882ef..0e9cc9a6fe867 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala @@ -17,11 +17,14 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving + /** * Describes the structure of a [[PropertyGraph]]. * * @since 3.0.0 */ +@Evolving trait PropertyGraphType { /** 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 index 3b71f763636ca..9fba6bcc1067a 100644 --- 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 @@ -19,8 +19,10 @@ 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 diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDataset.scala similarity index 93% rename from graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDataset.scala index 6046d013d3ec6..707a0a4b9d6f7 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrame.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDataset.scala @@ -17,6 +17,7 @@ package org.apache.spark.graph.api +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** @@ -32,14 +33,15 @@ import org.apache.spark.sql.{Dataset, Row} * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ -case class RelationshipFrame private[graph] ( +@Evolving +case class RelationshipDataset private[graph]( ds: Dataset[Row], idColumn: String, sourceIdColumn: String, targetIdColumn: String, relationshipType: String, properties: Map[String, String]) - extends GraphElementFrame { + extends GraphElementDataset { override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDatasetBuilder.scala similarity index 79% rename from graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDatasetBuilder.scala index ef5ad3d6253d7..e0aee0f02fc77 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipFrameBuilder.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/RelationshipDatasetBuilder.scala @@ -19,15 +19,17 @@ package org.apache.spark.graph.api import scala.collection.JavaConverters._ +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** - * Interface used to build a [[RelationshipFrame]]. + * A builder for [[RelationshipDataset]]. * * @param ds Dataset containing a single relationship in each row * @since 3.0.0 */ -final class RelationshipFrameBuilder(val ds: Dataset[Row]) { +@Evolving +final class RelationshipDatasetBuilder(val ds: Dataset[Row]) { private var idColumn: String = CypherSession.ID_COLUMN private var sourceIdColumn: String = CypherSession.SOURCE_ID_COLUMN @@ -39,7 +41,7 @@ final class RelationshipFrameBuilder(val ds: Dataset[Row]) { * @param idColumn column that contains the relationship identifier * @since 3.0.0 */ - def idColumn(idColumn: String): RelationshipFrameBuilder = { + def idColumn(idColumn: String): RelationshipDatasetBuilder = { if (idColumn.isEmpty) { throw new IllegalArgumentException("idColumn must not be empty") } @@ -51,7 +53,7 @@ final class RelationshipFrameBuilder(val ds: Dataset[Row]) { * @param sourceIdColumn column that contains the source node identifier of the relationship * @since 3.0.0 */ - def sourceIdColumn(sourceIdColumn: String): RelationshipFrameBuilder = { + def sourceIdColumn(sourceIdColumn: String): RelationshipDatasetBuilder = { if (sourceIdColumn.isEmpty) { throw new IllegalArgumentException("sourceIdColumn must not be empty") } @@ -63,7 +65,7 @@ final class RelationshipFrameBuilder(val ds: Dataset[Row]) { * @param targetIdColumn column that contains the target node identifier of the relationship * @since 3.0.0 */ - def targetIdColumn(targetIdColumn: String): RelationshipFrameBuilder = { + def targetIdColumn(targetIdColumn: String): RelationshipDatasetBuilder = { if (targetIdColumn.isEmpty) { throw new IllegalArgumentException("targetIdColumn must not be empty") } @@ -75,7 +77,7 @@ final class RelationshipFrameBuilder(val ds: Dataset[Row]) { * @param relationshipType relationship type that is assigned to all relationships * @since 3.0.0 */ - def relationshipType(relationshipType: String): RelationshipFrameBuilder = { + def relationshipType(relationshipType: String): RelationshipDatasetBuilder = { if (relationshipType.isEmpty) { throw new IllegalArgumentException("Relationship type must not be empty") } @@ -87,7 +89,7 @@ final class RelationshipFrameBuilder(val ds: Dataset[Row]) { * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def properties(properties: Map[String, String]): RelationshipFrameBuilder = { + def properties(properties: Map[String, String]): RelationshipDatasetBuilder = { this.properties = properties this } @@ -96,20 +98,20 @@ final class RelationshipFrameBuilder(val ds: Dataset[Row]) { * @param properties mapping from property keys to corresponding columns * @since 3.0.0 */ - def properties(properties: java.util.Map[String, String]): RelationshipFrameBuilder = { + def properties(properties: java.util.Map[String, String]): RelationshipDatasetBuilder = { this.properties = properties.asScala.toMap this } /** - * Creates a [[RelationshipFrame]] from the specified builder parameters. + * Creates a [[RelationshipDataset]] from the specified builder parameters. * * @since 3.0.0 */ - def build(): RelationshipFrame = { + def build(): RelationshipDataset = { maybeRelationshipType match { case Some(relType) => - RelationshipFrame(ds, idColumn, sourceIdColumn, targetIdColumn, relType, properties) + 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 index 5fd09213ae426..df986a516eaf4 100644 --- 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 @@ -64,7 +64,7 @@ public void tearDown() { } @Test - public void testCreateFromNodeFrame() { + public void testCreateFromNodeDataset() { StructType personSchema = createSchema( Lists.newArrayList("id", "name"), Lists.newArrayList(LongType, StringType)); @@ -80,14 +80,14 @@ public void testCreateFromNodeFrame() { List knowsData = Collections.singletonList(RowFactory.create(0L, 0L, 1L, 1984)); Dataset personDf = spark.createDataFrame(personData, personSchema); - NodeFrame personNodeFrame = cypherSession.buildNodeFrame(personDf) + NodeDataset personNodeDataset = cypherSession.buildNodeDataset(personDf) .idColumn("id") .labelSet(new String[]{"Person"}) .properties(Collections.singletonMap("name", "name")) .build(); Dataset knowsDf = spark.createDataFrame(knowsData, knowsSchema); - RelationshipFrame knowsRelFrame = cypherSession.buildRelationshipFrame(knowsDf) + RelationshipDataset knowsRelDataset = cypherSession.buildRelationshipDataset(knowsDf) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") @@ -97,8 +97,8 @@ public void testCreateFromNodeFrame() { PropertyGraph graph = cypherSession.createGraph( - new NodeFrame[]{personNodeFrame}, - new RelationshipFrame[]{knowsRelFrame}); + new NodeDataset[]{personNodeDataset}, + new RelationshipDataset[]{knowsRelDataset}); List result = graph.nodes().collectAsList(); Assert.assertEquals(2, result.size()); } 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 index 970b6f41fea98..0ee08823d50b5 100644 --- a/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala +++ b/graph/cypher/src/main/scala/org/apache/spark/cypher/SparkCypherSession.scala @@ -72,11 +72,11 @@ private[spark] class SparkCypherSession(override val sparkSession: SparkSession) throw UnsupportedOperationException("Graph construction with `CONSTRUCT` is not supported in Cypher 9") } - override def createGraph(nodes: Array[NodeFrame], relationships: Array[RelationshipFrame]): PropertyGraph = { + override def createGraph(nodes: Array[NodeDataset], relationships: Array[RelationshipDataset]): PropertyGraph = { require(nodes.groupBy(_.labelSet).forall(_._2.length == 1), - "There can be at most one NodeFrame per label set") + "There can be at most one NodeDataset per label set") require(relationships.groupBy(_.relationshipType).forall(_._2.length == 1), - "There can be at most one RelationshipFrame per relationship type") + "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))) 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 index 2444eb5f07f10..e02e3331f0c71 100644 --- 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 @@ -18,12 +18,12 @@ package org.apache.spark.cypher.adapters -import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} +import org.apache.spark.graph.api.{NodeDataset, RelationshipDataset} import org.opencypher.okapi.api.io.conversion.{ElementMapping, NodeMappingBuilder, RelationshipMappingBuilder} object MappingAdapter { - implicit class RichNodeDataFrame(val nodeDf: NodeFrame) extends AnyVal { + implicit class RichNodeDataDataset(val nodeDf: NodeDataset) extends AnyVal { def toNodeMapping: ElementMapping = NodeMappingBuilder .on(nodeDf.idColumn) .withImpliedLabels(nodeDf.labelSet.toSeq: _*) @@ -31,7 +31,7 @@ object MappingAdapter { .build } - implicit class RichRelationshipDataFrame(val relDf: RelationshipFrame) extends AnyVal { + implicit class RichRelationshipDataDataset(val relDf: RelationshipDataset) extends AnyVal { def toRelationshipMapping: ElementMapping = RelationshipMappingBuilder .on(relDf.idColumn) .withSourceStartNodeKey(relDf.sourceIdColumn) 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 index 2027efe9844f2..cf26d5b2a0089 100644 --- 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 @@ -22,15 +22,15 @@ 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.{NodeFrame, PropertyGraph, PropertyGraphType, RelationshipFrame} +import org.apache.spark.graph.api.{NodeDataset, PropertyGraph, PropertyGraphType, 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[NodeFrame], - relationshipFrames: Seq[RelationshipFrame]) extends PropertyGraph { + nodeFrames: Seq[NodeDataset], + relationshipFrames: Seq[RelationshipDataset]) extends PropertyGraph { override def schema: PropertyGraphType = SchemaAdapter(graph.schema) @@ -38,15 +38,15 @@ case class RelationalGraphAdapter( if (nodeFrames.isEmpty) { cypherSession.graphs.empty } else { - val nodeTables = nodeFrames.map { nodeDataFrame => SparkEntityTable(nodeDataFrame.toNodeMapping, nodeDataFrame.ds) } - val relTables = relationshipFrames.map { relDataFrame => SparkEntityTable(relDataFrame.toRelationshipMapping, relDataFrame.ds) } + 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], NodeFrame] = nodeFrames.map(nf => nf.labelSet -> nf).toMap + private lazy val _nodeFrame: Map[Set[String], NodeDataset] = nodeFrames.map(nf => nf.labelSet -> nf).toMap - private lazy val _relationshipFrame: Map[String, RelationshipFrame] = relationshipFrames.map(rf => rf.relationshipType -> rf).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 @@ -84,9 +84,9 @@ case class RelationalGraphAdapter( df.select(selectColumns: _*) } - override def nodeFrame(labelSet: Array[String]): NodeFrame = _nodeFrame(labelSet.toSet) + override def nodeDataset(labelSet: Array[String]): NodeDataset = _nodeFrame(labelSet.toSet) - override def relationshipFrame(relationshipType: String): RelationshipFrame = _relationshipFrame(relationshipType) + override def relationshipDataset(relationshipType: String): RelationshipDataset = _relationshipFrame(relationshipType) override def write(): SparkCypherPropertyGraphWriter = 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 index bbe131bf4c565..e0d2fbff28b06 100644 --- 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 @@ -18,13 +18,13 @@ package org.apache.spark.cypher.conversions -import org.apache.spark.graph.api.GraphElementFrame +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: GraphElementFrame): Dataset[Row] = { + def normalizeDf(frame: GraphElementDataset): Dataset[Row] = { val mappedColumnNames = frame.idColumns ++ frame.properties.values.toSeq.sorted val mappedDf = if (mappedColumnNames == frame.ds.columns.toSeq) { frame.ds 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 index 22def6e1ef4d4..28c85cd19bde6 100644 --- 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 @@ -25,7 +25,7 @@ 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.{NodeFrame, RelationshipFrame} +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} @@ -49,13 +49,13 @@ object ReadWriteGraph { } } - def nodeFrames: Seq[NodeFrame] = { + 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 - NodeFrame( + NodeDataset( df, SourceIdKey.name, combo, @@ -63,13 +63,13 @@ object ReadWriteGraph { } } - def relationshipFrames: Seq[RelationshipFrame] = { + 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 - RelationshipFrame( + RelationshipDataset( df, SourceIdKey.name, SourceStartNodeKey.name, 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 index 01c64232ced7c..b183c78d0be3b 100644 --- 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 @@ -25,7 +25,7 @@ class SparkCypherPropertyGraphReader(session: CypherSession) extends PropertyGra override def load(path: String): PropertyGraph = { val graphImporter = GraphImporter(session.sparkSession, path, format) - session.createGraph(graphImporter.nodeFrames.toArray, graphImporter.relationshipFrames.toArray) + session.createGraph(graphImporter.nodeDatasets.toArray, graphImporter.relationshipDatasets.toArray) } } 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 index 60de6f1a89a04..a4ac474c10f31 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -19,38 +19,38 @@ package org.apache.spark.cypher import org.apache.spark.SparkFunSuite -import org.apache.spark.graph.api.{CypherResult, NodeFrame, PropertyGraph, RelationshipFrame} +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 NodeFrame") { + test("create PropertyGraph from single NodeDataset") { val nodeData: Dataset[Row] = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")).toDF("id", "name") - val nodeFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) + val nodeDataset: NodeDataset = cypherSession.buildNodeDataset(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) + 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 RelationshipFrames") { + 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 nodeFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) + val nodeDataset: NodeDataset = cypherSession.buildNodeDataset(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val relationshipFrame: RelationshipFrame = cypherSession.buildRelationshipFrame(relationshipData) + val relationshipDataset: RelationshipDataset = cypherSession.buildRelationshipDataset(relationshipData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("KNOWS") .build() - val graph: PropertyGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Array(nodeDataset), Array(relationshipDataset)) val result: CypherResult = graph.cypher( """ |MATCH (a:Person)-[r:KNOWS]->(:Person) @@ -59,16 +59,16 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { } test("create PropertyGraph with multiple node and relationship types") { - val studentDF: Dataset[Row] = spark.createDataFrame(Seq((0, "Alice", 42), (1, "Bob", 23))).toDF("id", "name", "age") - val teacherDF: Dataset[Row] = spark.createDataFrame(Seq((2, "Eve", "CS"))).toDF("id", "name", "subject") + 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 = cypherSession.buildNodeFrame(studentDF) + val studentNF = cypherSession.buildNodeDataset(studentDS) .idColumn("id") .labelSet(Array("Person", "Student")) .properties(Map("name" -> "name", "age" -> "age")) .build() - val teacherNF = cypherSession.buildNodeFrame(teacherDF) + val teacherNF = cypherSession.buildNodeDataset(teacherDS) .idColumn("id") .labelSet(Array("Person", "Teacher")) .properties(Map("name" -> "name", "subject" -> "subject")) @@ -77,14 +77,14 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { 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: RelationshipFrame = cypherSession.buildRelationshipFrame(knowsDF) + val knowsRF: RelationshipDataset = cypherSession.buildRelationshipDataset(knowsDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("KNOWS") .properties(Map("since" -> "since")) .build() - val teachesRF: RelationshipFrame = cypherSession.buildRelationshipFrame(teachesDF) + val teachesRF: RelationshipDataset = cypherSession.buildRelationshipDataset(teachesDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") @@ -100,14 +100,14 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { 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: NodeFrame = NodeFrame(studentDF, "id", Set("Person", "Student"), Map("name" -> "col_name", "age" -> "col_age")) - val teacherNF: NodeFrame = NodeFrame(teacherDF, "id", Set("Person", "Teacher"), Map("name" -> "col_name", "subject" -> "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: RelationshipFrame = RelationshipFrame(knowsDF, "id", "source", "target", "KNOWS", Map("since" -> "col_since")) - val teachesRF: RelationshipFrame = RelationshipFrame(teachesDF, "id", "source", "target", "TEACHES", Map.empty) + 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") 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 index b3eabfcce829e..72eb34d4ed022 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -20,7 +20,7 @@ package org.apache.spark.cypher import java.nio.file.Paths -import org.apache.spark.graph.api.{NodeFrame, RelationshipFrame} +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 @@ -51,13 +51,13 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef Tuple3(0, 0, 1) )).toDF("id", "source", "target") - private lazy val nodeDataFrame: NodeFrame = cypherSession.buildNodeFrame(nodeData) + private lazy val nodeDataFrame: NodeDataset = cypherSession.buildNodeDataset(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - private lazy val relationshipFrame: RelationshipFrame = RelationshipFrame(relationshipData, "id", "source", "target", "KNOWS", Map.empty) + 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)) @@ -71,7 +71,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef test("save and loads a property graph") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = cypherSession.buildNodeFrame(nodeData) + val nodeFrame = cypherSession.buildNodeDataset(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) @@ -80,7 +80,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) .toDF("id", "source", "target", "since") - val relationshipFrame = RelationshipFrame(relationshipData, "id", "source", "target", "KNOWS", Map.empty) + val relationshipFrame = RelationshipDataset(relationshipData, "id", "source", "target", "KNOWS", Map.empty) val writeGraph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) 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 index 7a2f4e2fabf1f..4bce54bb6a019 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.cypher import org.scalatest.Matchers -import org.apache.spark.graph.api.{CypherSession, NodeFrame, RelationshipFrame} +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 @@ -74,14 +74,14 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers (8L, 2L, 5L, false, true))) .toDF(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN, label("KNOWS"), label("STUDY_AT")) - test("create graph from NodeFrame") { + test("create graph from NodeDataset") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = cypherSession.buildNodeFrame(nodeData) + val nodeDataset = cypherSession.buildNodeDataset(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val graph = cypherSession.createGraph(Array(nodeFrame), Array.empty[RelationshipFrame]) + val graph = cypherSession.createGraph(Array(nodeDataset), Array.empty[RelationshipDataset]) val expectedDf = spark .createDataFrame(Seq((convertId(0L), true, "Alice"), (convertId(1L), true, "Bob"))) @@ -90,9 +90,9 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers checkAnswer(graph.nodes, expectedDf) } - test("create graph from NodeFrame and RelationshipFrame") { + test("create graph from NodeDataset and RelationshipDataset") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = cypherSession.buildNodeFrame(nodeData) + val nodeDataset = cypherSession.buildNodeDataset(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) @@ -100,15 +100,15 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) .toDF("id", "source", "target", "since") - val relationshipFrame = cypherSession.buildRelationshipFrame(relationshipData) - .idColumn("id") - .sourceIdColumn("source") - .targetIdColumn("target") - .relationshipType("KNOWS") - .properties(Map("since" -> "since")) - .build() + val relationshipDataset = cypherSession.buildRelationshipDataset(relationshipData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() - val graph = cypherSession.createGraph(Array(nodeFrame), Array(relationshipFrame)) + val graph = cypherSession.createGraph(Array(nodeDataset), Array(relationshipDataset)) val expectedNodeDf = spark .createDataFrame(Seq((convertId(0L), true, "Alice"), (convertId(1L), true, "Bob"))) @@ -130,13 +130,13 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((2L, "Eve", "CS"))) .toDF("id", "name", "subject") - val studentNF = cypherSession.buildNodeFrame(studentDF) + val studentNodeDataset = cypherSession.buildNodeDataset(studentDF) .idColumn("id") .labelSet(Array("Person", "Student")) .properties(Map("name" -> "name", "age" -> "age")) .build() - val teacherNF = cypherSession.buildNodeFrame(teacherDF) + val teacherNodeDataset = cypherSession.buildNodeDataset(teacherDF) .idColumn("id") .labelSet(Array("Person", "Teacher")) .properties(Map("name" -> "name", "subject" -> "subject")) @@ -149,21 +149,23 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((1L, 2L, 1L))) .toDF("id", "source", "target") - val knowsRF = cypherSession.buildRelationshipFrame(knowsDF) - .idColumn("id") - .sourceIdColumn("source") - .targetIdColumn("target") - .relationshipType("KNOWS") - .properties(Map("since" -> "since")) - .build() - val teachesRF = cypherSession.buildRelationshipFrame(teachesDF) + val knowsRelationshipDataset = cypherSession.buildRelationshipDataset(knowsDF) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("KNOWS") + .properties(Map("since" -> "since")) + .build() + val teachesRelationshipDataset = cypherSession.buildRelationshipDataset(teachesDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("TEACHES") .build() - val graph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) + val graph = cypherSession.createGraph( + Array(studentNodeDataset, teacherNodeDataset), + Array(knowsRelationshipDataset, teachesRelationshipDataset)) val expectedNodeDf = spark .createDataFrame( @@ -206,12 +208,12 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((2L, "Eve", "CS"))) .toDF("id", "col_name", "col_subject") - val studentNF = NodeFrame( + val studentNodeDataset = NodeDataset( studentDF, "id", Set("Person", "Student"), properties = Map("name" -> "col_name", "age" -> "col_age")) - val teacherNF = NodeFrame( + val teacherNodeDataset = NodeDataset( teacherDF, "id", Set("Person", "Teacher"), @@ -220,14 +222,14 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers 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 knowsRF = RelationshipFrame( + val knowsRelationshipDataset = RelationshipDataset( knowsDF, "id", "source", "target", relationshipType = "KNOWS", properties = Map("since" -> "col_since")) - val teachesRF = RelationshipFrame( + val teachesRelationshipDataset = RelationshipDataset( teachesDF, "id", "source", @@ -235,7 +237,9 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "TEACHES", Map.empty) - val graph = cypherSession.createGraph(Array(studentNF, teacherNF), Array(knowsRF, teachesRF)) + val graph = cypherSession.createGraph( + Array(studentNodeDataset, teacherNodeDataset), + Array(knowsRelationshipDataset, teachesRelationshipDataset)) val expectedNodeDf = spark .createDataFrame( @@ -272,28 +276,32 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("select nodes via label set") { val graph = cypherSession.createGraph(nodes, relationships) - val nodeFrame = graph.nodeFrame(Array("Person", "Teacher")) + val nodeDataset = graph.nodeDataset(Array("Person", "Teacher")) - nodeFrame.labelSet shouldEqual Set("Person", "Teacher") - nodeFrame.idColumn shouldEqual ID_COLUMN - nodeFrame.properties shouldEqual Map("age" -> "age", "name" -> "name", "subject" -> "subject", "title" -> "title") + nodeDataset.labelSet shouldEqual Set("Person", "Teacher") + nodeDataset.idColumn shouldEqual ID_COLUMN + nodeDataset.properties 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(nodeFrame.ds, expectedNodeDf) + checkAnswer(nodeDataset.ds, expectedNodeDf) } test("select relationships via type") { val graph = cypherSession.createGraph(nodes, relationships) - val relationshipFrame = graph.relationshipFrame("KNOWS") + val relationshipDataset = graph.relationshipDataset("KNOWS") - relationshipFrame.relationshipType shouldEqual "KNOWS" - relationshipFrame.idColumn shouldEqual ID_COLUMN - relationshipFrame.sourceIdColumn shouldEqual SOURCE_ID_COLUMN - relationshipFrame.targetIdColumn shouldEqual TARGET_ID_COLUMN - relationshipFrame.properties shouldBe empty + relationshipDataset.relationshipType shouldEqual "KNOWS" + relationshipDataset.idColumn shouldEqual ID_COLUMN + relationshipDataset.sourceIdColumn shouldEqual SOURCE_ID_COLUMN + relationshipDataset.targetIdColumn shouldEqual TARGET_ID_COLUMN + relationshipDataset.properties shouldBe empty val expectedRelDf = spark .createDataFrame( @@ -304,8 +312,8 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers (convertId(3L), convertId(3L), convertId(0L)), (convertId(4L), convertId(3L), convertId(1L)))) .toDF(ID_COLUMN, SOURCE_ID_COLUMN, TARGET_ID_COLUMN) - - checkAnswer(relationshipFrame.ds, expectedRelDf) + + checkAnswer(relationshipDataset.ds, expectedRelDf) } private def label(label: String): String = s"$LABEL_COLUMN_PREFIX$label" From 9114585788ddab56d26ed19d31924c3784aa3f87 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Mon, 14 Oct 2019 19:24:22 +0200 Subject: [PATCH 109/123] Address review comments --- .../scala/org/apache/spark/graph/api/CypherSession.scala | 8 ++++---- .../org/apache/spark/graph/api/GraphElementDataset.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) 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 index 8de5baf7e91fc..abc4662ec8a17 100644 --- 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 @@ -59,7 +59,7 @@ object CypherSession { * @param nodes node dataset * @since 3.0.0 */ - def extractNodeDataset(nodes: Dataset[Row]): Set[NodeDataset] = { + def extractNodeDatasets(nodes: Dataset[Row]): Set[NodeDataset] = { val labelColumns = nodes.columns.filter(_.startsWith(LABEL_COLUMN_PREFIX)).toSet validateLabelColumns(nodes.schema, labelColumns) @@ -102,7 +102,7 @@ object CypherSession { * @param relationships relationship dataset * @since 3.0.0 */ - def extractRelationshipDataset(relationships: Dataset[Row]): Set[RelationshipDataset] = { + def extractRelationshipDatasets(relationships: Dataset[Row]): Set[RelationshipDataset] = { val relColumns = relationships.columns.toSet val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) validateLabelColumns(relationships.schema, relTypeColumns) @@ -251,8 +251,8 @@ trait CypherSession { * @since 3.0.0 */ def createGraph(nodes: Dataset[Row], relationships: Dataset[Row]): PropertyGraph = { - val nodeFrames = CypherSession.extractNodeDataset(nodes) - val relationshipFrames = CypherSession.extractRelationshipDataset(relationships) + val nodeFrames = CypherSession.extractNodeDatasets(nodes) + val relationshipFrames = CypherSession.extractRelationshipDatasets(relationships) createGraph(nodeFrames.toArray, relationshipFrames.toArray) } 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 index 5f84b8fbc0bc9..7ba3c1eb1a23f 100644 --- 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 @@ -21,7 +21,7 @@ import org.apache.spark.annotation.Evolving import org.apache.spark.sql.{Dataset, Row} /** - * A [[PropertyGraph]] whose graph element is either a node or a relationship. + * 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 From c337f6cb4673ef438829edcc782e891cbbe27436 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 15 Oct 2019 00:05:38 +0200 Subject: [PATCH 110/123] Address review comments --- .../spark/graph/api/CypherSession.scala | 35 +++++++++++++------ .../spark/graph/api/GraphElementDataset.scala | 4 +-- .../apache/spark/graph/api/NodeDataset.scala | 10 +++--- .../spark/graph/api/PropertyGraph.scala | 2 +- ...phType.scala => PropertyGraphSchema.scala} | 8 ++--- .../spark/graph/api/RelationshipDataset.scala | 6 ++-- .../cypher/adapters/MappingAdapter.scala | 4 +-- .../adapters/RelationalGraphAdapter.scala | 4 +-- .../spark/cypher/adapters/SchemaAdapter.scala | 10 +++--- .../GraphElementFrameConversions.scala | 2 +- .../spark/cypher/PropertyGraphSuite.scala | 10 +++--- 11 files changed, 54 insertions(+), 41 deletions(-) rename graph/api/src/main/scala/org/apache/spark/graph/api/{PropertyGraphType.scala => PropertyGraphSchema.scala} (89%) 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 index abc4662ec8a17..3874cca6e28a3 100644 --- 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 @@ -47,10 +47,15 @@ object CypherSession { val TARGET_ID_COLUMN = "$TARGET_ID" /** - * Naming convention both for node label and relationship type prefixes. + * 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. * @@ -59,9 +64,9 @@ object CypherSession { * @param nodes node dataset * @since 3.0.0 */ - def extractNodeDatasets(nodes: Dataset[Row]): Set[NodeDataset] = { + def extractNodeDatasets(nodes: Dataset[Row]): Array[NodeDataset] = { val labelColumns = nodes.columns.filter(_.startsWith(LABEL_COLUMN_PREFIX)).toSet - validateLabelColumns(nodes.schema, labelColumns) + validateTypeColumns(nodes.schema, labelColumns, LABEL_COLUMN_PREFIX) val nodeProperties = (nodes.columns.toSet - ID_COLUMN -- labelColumns) .map(col => col -> col) @@ -70,7 +75,7 @@ object CypherSession { val labelCount = labelColumns.size if (labelCount > 5) { LoggerFactory.getLogger(CypherSession.getClass).warn( - s"$labelCount label columns will result in ${Math.pow(labelCount, 2)} node frames.") + 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.") @@ -91,7 +96,7 @@ object CypherSession { .reduce(_ && _) NodeDataset(nodes.filter(predicate), ID_COLUMN, labelSet.map(_.substring(1)), nodeProperties) - } + }.toArray } /** @@ -102,10 +107,10 @@ object CypherSession { * @param relationships relationship dataset * @since 3.0.0 */ - def extractRelationshipDatasets(relationships: Dataset[Row]): Set[RelationshipDataset] = { + def extractRelationshipDatasets(relationships: Dataset[Row]): Array[RelationshipDataset] = { val relColumns = relationships.columns.toSet - val relTypeColumns = relColumns.filter(_.startsWith(CypherSession.LABEL_COLUMN_PREFIX)) - validateLabelColumns(relationships.schema, relTypeColumns) + val relTypeColumns = relColumns.filter(_.startsWith(REL_TYPE_COLUMN_PREFIX)) + validateTypeColumns(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 @@ -119,15 +124,23 @@ object CypherSession { TARGET_ID_COLUMN, relTypeColumn.substring(1), relProperties) - } + }.toArray } - private def validateLabelColumns(schema: StructType, columns: Set[String]): Unit = { + private def validateTypeColumns( + 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.") + } + }) } } @@ -253,7 +266,7 @@ trait CypherSession { def createGraph(nodes: Dataset[Row], relationships: Dataset[Row]): PropertyGraph = { val nodeFrames = CypherSession.extractNodeDatasets(nodes) val relationshipFrames = CypherSession.extractRelationshipDatasets(relationships) - createGraph(nodeFrames.toArray, relationshipFrames.toArray) + createGraph(nodeFrames, relationshipFrames) } /** 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 index 7ba3c1eb1a23f..dd7d915f75fb6 100644 --- 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 @@ -48,7 +48,7 @@ abstract class GraphElementDataset { * * @since 3.0.0 */ - def idColumns: Seq[String] = Seq(idColumn) + def idColumns: Array[String] = Array(idColumn) /** * Mapping from graph element property keys to the columns that contain the corresponding property @@ -56,6 +56,6 @@ abstract class GraphElementDataset { * * @since 3.0.0 */ - def properties: Map[String, String] + 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 index b87504d356de2..08df9aa182273 100644 --- 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 @@ -26,10 +26,10 @@ import org.apache.spark.sql.{Dataset, Row} * 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 properties mapping from property keys to corresponding columns + * @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 @@ -37,5 +37,5 @@ case class NodeDataset private[graph]( ds: Dataset[Row], idColumn: String, labelSet: Set[String], - properties: Map[String, String]) + propertyColumns: Map[String, String]) extends GraphElementDataset 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 index 9f7f179ee1ef5..de58187472129 100644 --- 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 @@ -38,7 +38,7 @@ abstract class PropertyGraph { * * @since 3.0.0 */ - def schema: PropertyGraphType + def schema: PropertyGraphSchema /** * The session in which this graph is managed. diff --git a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphSchema.scala similarity index 89% rename from graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala rename to graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphSchema.scala index 0e9cc9a6fe867..52c690fb71a6e 100644 --- a/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphType.scala +++ b/graph/api/src/main/scala/org/apache/spark/graph/api/PropertyGraphSchema.scala @@ -25,26 +25,26 @@ import org.apache.spark.annotation.Evolving * @since 3.0.0 */ @Evolving -trait PropertyGraphType { +trait PropertyGraphSchema { /** * Returns all labels occurring on any node in the graph. * * @since 3.0.0 */ - def labels: Set[String] = labelSets.flatten + def labels: Array[String] = labelSets.flatten /** * Returns all distinct label sets occurring on nodes in the graph. * * @since 3.0.0 */ - def labelSets: Set[Set[String]] + def labelSets: Array[Array[String]] /** * Returns all relationship types occurring on relationships in the graph. * * @since 3.0.0 */ - def relationshipTypes: Set[String] + def relationshipTypes: Array[String] } 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 index 707a0a4b9d6f7..711acbc17a36b 100644 --- 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 @@ -30,7 +30,7 @@ import org.apache.spark.sql.{Dataset, Row} * @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 properties mapping from property keys to corresponding columns + * @param propertyColumns mapping from property keys to corresponding columns * @since 3.0.0 */ @Evolving @@ -40,9 +40,9 @@ case class RelationshipDataset private[graph]( sourceIdColumn: String, targetIdColumn: String, relationshipType: String, - properties: Map[String, String]) + propertyColumns: Map[String, String]) extends GraphElementDataset { - override def idColumns: Seq[String] = Seq(idColumn, sourceIdColumn, targetIdColumn) + override def idColumns: Array[String] = Array(idColumn, sourceIdColumn, targetIdColumn) } 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 index e02e3331f0c71..d2eea2b06c013 100644 --- 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 @@ -27,7 +27,7 @@ object MappingAdapter { def toNodeMapping: ElementMapping = NodeMappingBuilder .on(nodeDf.idColumn) .withImpliedLabels(nodeDf.labelSet.toSeq: _*) - .withPropertyKeyMappings(nodeDf.properties.toSeq:_*) + .withPropertyKeyMappings(nodeDf.propertyColumns.toSeq:_*) .build } @@ -37,7 +37,7 @@ object MappingAdapter { .withSourceStartNodeKey(relDf.sourceIdColumn) .withSourceEndNodeKey(relDf.targetIdColumn) .withRelType(relDf.relationshipType) - .withPropertyKeyMappings(relDf.properties.toSeq: _*) + .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 index cf26d5b2a0089..8e36c52346709 100644 --- 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 @@ -22,7 +22,7 @@ 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, PropertyGraphType, RelationshipDataset} +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 @@ -32,7 +32,7 @@ case class RelationalGraphAdapter( nodeFrames: Seq[NodeDataset], relationshipFrames: Seq[RelationshipDataset]) extends PropertyGraph { - override def schema: PropertyGraphType = SchemaAdapter(graph.schema) + override def schema: PropertyGraphSchema = SchemaAdapter(graph.schema) private[cypher] lazy val graph = { if (nodeFrames.isEmpty) { 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 index 5450c915d89dc..0a65d1135aed9 100644 --- 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 @@ -18,14 +18,14 @@ package org.apache.spark.cypher.adapters -import org.apache.spark.graph.api.PropertyGraphType -import org.opencypher.okapi.api.schema.PropertyGraphSchema +import org.apache.spark.graph.api.PropertyGraphSchema +import org.opencypher.okapi.api.schema.{PropertyGraphSchema => OKAPISchema} -case class SchemaAdapter(schema: PropertyGraphSchema) extends PropertyGraphType { +case class SchemaAdapter(schema: OKAPISchema) extends PropertyGraphSchema { - override def labelSets: Set[Set[String]] = schema.labelCombinations.combos + override def labelSets: Array[Array[String]] = schema.labelCombinations.combos.map(_.toArray).toArray - override def relationshipTypes: Set[String] = schema.relationshipTypes + override def relationshipTypes: Array[String] = schema.relationshipTypes.toArray } 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 index e0d2fbff28b06..99c69fe147c4e 100644 --- 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 @@ -25,7 +25,7 @@ import org.apache.spark.sql.{Dataset, Row} object GraphElementFrameConversions { def normalizeDf(frame: GraphElementDataset): Dataset[Row] = { - val mappedColumnNames = frame.idColumns ++ frame.properties.values.toSeq.sorted + val mappedColumnNames = frame.idColumns.toSeq ++ frame.propertyColumns.values.toSeq.sorted val mappedDf = if (mappedColumnNames == frame.ds.columns.toSeq) { frame.ds } else { 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 index 4bce54bb6a019..9f764c33fb9bf 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -212,12 +212,12 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers studentDF, "id", Set("Person", "Student"), - properties = Map("name" -> "col_name", "age" -> "col_age")) + propertyColumns = Map("name" -> "col_name", "age" -> "col_age")) val teacherNodeDataset = NodeDataset( teacherDF, "id", Set("Person", "Teacher"), - properties = Map("name" -> "col_name", "subject" -> "col_subject")) + 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") @@ -228,7 +228,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers "source", "target", relationshipType = "KNOWS", - properties = Map("since" -> "col_since")) + propertyColumns = Map("since" -> "col_since")) val teachesRelationshipDataset = RelationshipDataset( teachesDF, "id", @@ -280,7 +280,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers nodeDataset.labelSet shouldEqual Set("Person", "Teacher") nodeDataset.idColumn shouldEqual ID_COLUMN - nodeDataset.properties shouldEqual Map( + nodeDataset.propertyColumns shouldEqual Map( "age" -> "age", "name" -> "name", "subject" -> "subject", @@ -301,7 +301,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers relationshipDataset.idColumn shouldEqual ID_COLUMN relationshipDataset.sourceIdColumn shouldEqual SOURCE_ID_COLUMN relationshipDataset.targetIdColumn shouldEqual TARGET_ID_COLUMN - relationshipDataset.properties shouldBe empty + relationshipDataset.propertyColumns shouldBe empty val expectedRelDf = spark .createDataFrame( From 47948cd78db50c0ad68f36fc9829f5c1cd8e860c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Florentin=20D=C3=B6rre?= Date: Wed, 2 Oct 2019 19:46:02 +0200 Subject: [PATCH 111/123] Add a simple spark-graph example --- examples/pom.xml | 12 ++++ .../spark/examples/graph/SimpleDemo.scala | 61 +++++++++++++++++++ 2 files changed, 73 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.scala diff --git a/examples/pom.xml b/examples/pom.xml index a099f1e042e99..5175e2640b171 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -89,6 +89,18 @@ ${project.version} provided
+ + org.apache.spark + spark-graph-api_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-cypher_${scala.binary.version} + ${project.version} + provided + org.apache.commons commons-math3 diff --git a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.scala b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.scala new file mode 100644 index 0000000000000..485582227c318 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.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.examples.graph + +// $example on$ + +import org.apache.spark.cypher.SparkCypherSession +// $example off$ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.SparkSession + +object SimpleDemo { + + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder() + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // Initialise a GraphSession + val cypherSession = SparkCypherSession.create(spark) + + // Create node df and edge df + val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")) + .toDF("id", "name") + val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))) + .toDF("id", "source", "target") + + // Create a PropertyGraph + val graph = cypherSession.createGraph(nodeData, relationshipData) + + // Run our first query + val result = graph.cypher( + """ + |MATCH (a:Person)-[r:KNOWS]->(:Person) + |RETURN a, r""".stripMargin) + + // Print the result + result.df.show() + + + spark.stop() + } +} + From 17b62144876f5b93556e14d19116afbe80e31844 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Florentin=20D=C3=B6rre?= Date: Fri, 4 Oct 2019 12:24:13 +0200 Subject: [PATCH 112/123] Add CompleteGraphAPIExample --- .../src/main/resources/mini_yelp/business.csv | 4 + .../src/main/resources/mini_yelp/reviews.csv | 4 + .../src/main/resources/mini_yelp/user.csv | 4 + .../graph/CompleteGraphAPIExample.scala | 80 +++++++++++++++++++ .../{SimpleDemo.scala => SimpleExample.scala} | 9 +-- 5 files changed, 96 insertions(+), 5 deletions(-) create mode 100644 examples/src/main/resources/mini_yelp/business.csv create mode 100644 examples/src/main/resources/mini_yelp/reviews.csv create mode 100644 examples/src/main/resources/mini_yelp/user.csv create mode 100644 examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala rename examples/src/main/scala/org/apache/spark/examples/graph/{SimpleDemo.scala => SimpleExample.scala} (98%) diff --git a/examples/src/main/resources/mini_yelp/business.csv b/examples/src/main/resources/mini_yelp/business.csv new file mode 100644 index 0000000000000..c4bbc7a086f36 --- /dev/null +++ b/examples/src/main/resources/mini_yelp/business.csv @@ -0,0 +1,4 @@ +id;name;cuisine +3;Mario's Pizza;Italian +4;Bob's Burger;American +5;HermannTheGerman;German diff --git a/examples/src/main/resources/mini_yelp/reviews.csv b/examples/src/main/resources/mini_yelp/reviews.csv new file mode 100644 index 0000000000000..b2a23d3795a51 --- /dev/null +++ b/examples/src/main/resources/mini_yelp/reviews.csv @@ -0,0 +1,4 @@ +id;user;business;rating +0;0;3;2 +1;1;5;5 +2;0;3;10 diff --git a/examples/src/main/resources/mini_yelp/user.csv b/examples/src/main/resources/mini_yelp/user.csv new file mode 100644 index 0000000000000..05882b07aa761 --- /dev/null +++ b/examples/src/main/resources/mini_yelp/user.csv @@ -0,0 +1,4 @@ +id;name +0;Alice +1;Bob +2;Eve 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..bc3972fb90cd2 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -0,0 +1,80 @@ +/* + * 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.{NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.sql.{SaveMode, SparkSession} +// $example off$ + +object CompleteGraphAPIExample { + def main(args: Array[String]): Unit = { + // Creates a SparkSession. + val spark = SparkSession + .builder() + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + val sc = spark.sparkContext + + // Load node dfs and edge df + val userData = spark.read.csv("examples/src/main/resources/mini_yelp/user.csv") + val businessData = spark.read.csv("examples/src/main/resources/mini_yelp/business.csv") + val reviewData = spark.read.csv("examples/src/main/resources/mini_yelp/review.csv") + + // Initialise a GraphSession + val cypherSession = SparkCypherSession.create(spark) + + // Create Node- and RelationshipFrames + val userNodeFrame = NodeFrame.create(userData, "id", Set("USER")) + val businessNodeFrame = NodeFrame.create(businessData, "id", Set("BUSINESS")) + val reviewRelationshipFrame = RelationshipFrame.create(reviewData, "id", "user", "business", "REVIEWS") + + + // Create a PropertyGraph + val graph: PropertyGraph = cypherSession.createGraph(Array(userNodeFrame, businessNodeFrame), Array(reviewRelationshipFrame)) + + // Get existing node labels + val labelSet = graph.schema.labels + print(s"The graph contains nodes with the following labels: ${labelSet.mkString(",")}") + + val businessNodes = graph.nodeFrame(Array("Business")) + businessNodes.df.show() + + // Run parameterized cypher query + val result = graph.cypher( + """ + |MATCH (a:USER)-[r:REVIEWS]->(b:BUSINESS) + |WHERE a.name = $name + |RETURN a, r.rating, b.name""".stripMargin, Map("name" -> "Bob")) + + print("Reviews from Bob") + result.df.show() + + // Store the PropertyGraph + val savePath = "examples/src/main/resources/exampleGraph/" + graph.save(savePath, SaveMode.Overwrite) + + // Load the PropertyGraph + val importedGraph = cypherSession.load(savePath) + + + spark.stop() + } +} + diff --git a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.scala b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.scala rename to examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala index 485582227c318..a652fa8ea2903 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleDemo.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala @@ -23,8 +23,7 @@ import org.apache.spark.cypher.SparkCypherSession import org.apache.spark.sql.DataFrame import org.apache.spark.sql.SparkSession -object SimpleDemo { - +object SimpleExample { def main(args: Array[String]): Unit = { // Creates a SparkSession. val spark = SparkSession @@ -33,15 +32,15 @@ object SimpleDemo { .getOrCreate() val sc = spark.sparkContext - // Initialise a GraphSession - val cypherSession = SparkCypherSession.create(spark) - // Create node df and edge df val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")) .toDF("id", "name") val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))) .toDF("id", "source", "target") + // Initialise a GraphSession + val cypherSession = SparkCypherSession.create(spark) + // Create a PropertyGraph val graph = cypherSession.createGraph(nodeData, relationshipData) From a7be7041120ffdc39a9321781b15f6ab6a553def Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Florentin=20D=C3=B6rre?= Date: Fri, 11 Oct 2019 09:17:31 +0200 Subject: [PATCH 113/123] Merge pull request #1 from knutwalker/spark_graph_examples Make example work when started from the IDE --- examples/pom.xml | 160 ++++++++++++++++-- .../graph/CompleteGraphAPIExample.scala | 20 ++- 2 files changed, 159 insertions(+), 21 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 5175e2640b171..22f94c4f76767 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -51,60 +51,66 @@ 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-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} - provided + compile org.apache.spark spark-cypher_${scala.binary.version} ${project.version} - provided + compile org.apache.commons commons-math3 - provided + compile org.scalacheck @@ -114,17 +120,145 @@ 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 @@ -163,7 +297,7 @@ org.apache.spark spark-streaming-kinesis-asl_${scala.binary.version} ${project.version} - provided + compile 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 index bc3972fb90cd2..5b79212888125 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -29,20 +29,23 @@ object CompleteGraphAPIExample { 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" -> ";") // Load node dfs and edge df - val userData = spark.read.csv("examples/src/main/resources/mini_yelp/user.csv") - val businessData = spark.read.csv("examples/src/main/resources/mini_yelp/business.csv") - val reviewData = spark.read.csv("examples/src/main/resources/mini_yelp/review.csv") + val userData = spark.read.options(csvConfig).csv("examples/src/main/resources/mini_yelp/user.csv") + val businessData = spark.read.options(csvConfig).csv("examples/src/main/resources/mini_yelp/business.csv") + val reviewData = spark.read.options(csvConfig).csv("examples/src/main/resources/mini_yelp/reviews.csv") // Initialise a GraphSession val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val userNodeFrame = NodeFrame.create(userData, "id", Set("USER")) - val businessNodeFrame = NodeFrame.create(businessData, "id", Set("BUSINESS")) + val userNodeFrame = NodeFrame.create(userData, "id", Set("User")) + val businessNodeFrame = NodeFrame.create(businessData, "id", Set("Business")) val reviewRelationshipFrame = RelationshipFrame.create(reviewData, "id", "user", "business", "REVIEWS") @@ -51,7 +54,8 @@ object CompleteGraphAPIExample { // Get existing node labels val labelSet = graph.schema.labels - print(s"The graph contains nodes with the following labels: ${labelSet.mkString(",")}") + println(s"The graph contains nodes with the following labels: ${labelSet.mkString(",")}") + println() val businessNodes = graph.nodeFrame(Array("Business")) businessNodes.df.show() @@ -59,11 +63,11 @@ object CompleteGraphAPIExample { // Run parameterized cypher query val result = graph.cypher( """ - |MATCH (a:USER)-[r:REVIEWS]->(b:BUSINESS) + |MATCH (a:User)-[r:REVIEWS]->(b:Business) |WHERE a.name = $name |RETURN a, r.rating, b.name""".stripMargin, Map("name" -> "Bob")) - print("Reviews from Bob") + println("Reviews from Bob") result.df.show() // Store the PropertyGraph From b8018bbb3008d535371e449e0653a36af0ec0de4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Florentin=20D=C3=B6rre?= Date: Tue, 8 Oct 2019 11:00:45 +0200 Subject: [PATCH 114/123] Add pauses in complete graph API example --- .../apache/spark/examples/graph/CompleteGraphAPIExample.scala | 4 ++++ 1 file changed, 4 insertions(+) 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 index 5b79212888125..a77a39c998a33 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -21,6 +21,8 @@ package org.apache.spark.examples.graph import org.apache.spark.cypher.SparkCypherSession import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} import org.apache.spark.sql.{SaveMode, SparkSession} + +import scala.io.StdIn // $example off$ object CompleteGraphAPIExample { @@ -56,9 +58,11 @@ object CompleteGraphAPIExample { 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.nodeFrame(Array("Business")) businessNodes.df.show() + StdIn.readLine("Press Enter to continue: ") // Run parameterized cypher query val result = graph.cypher( From ae4d457c29d136dd426b605d8309ad4de6121e79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Florentin=20D=C3=B6rre?= Date: Tue, 8 Oct 2019 12:07:13 +0200 Subject: [PATCH 115/123] Rework complete graph API example to movies dataset --- .../src/main/resources/mini_yelp/business.csv | 4 - .../src/main/resources/mini_yelp/reviews.csv | 4 - .../src/main/resources/mini_yelp/user.csv | 4 - .../src/main/resources/movies/acted_in.csv | 173 ++++++++++++++++++ .../src/main/resources/movies/directed.csv | 45 +++++ .../src/main/resources/movies/follows.csv | 4 + examples/src/main/resources/movies/movies.csv | 39 ++++ .../src/main/resources/movies/persons.csv | 134 ++++++++++++++ .../src/main/resources/movies/produced.csv | 16 ++ .../src/main/resources/movies/reviewed.csv | 10 + examples/src/main/resources/movies/wrote.csv | 11 ++ .../graph/CompleteGraphAPIExample.scala | 31 ++-- 12 files changed, 448 insertions(+), 27 deletions(-) delete mode 100644 examples/src/main/resources/mini_yelp/business.csv delete mode 100644 examples/src/main/resources/mini_yelp/reviews.csv delete mode 100644 examples/src/main/resources/mini_yelp/user.csv create mode 100644 examples/src/main/resources/movies/acted_in.csv create mode 100644 examples/src/main/resources/movies/directed.csv create mode 100644 examples/src/main/resources/movies/follows.csv create mode 100644 examples/src/main/resources/movies/movies.csv create mode 100644 examples/src/main/resources/movies/persons.csv create mode 100644 examples/src/main/resources/movies/produced.csv create mode 100644 examples/src/main/resources/movies/reviewed.csv create mode 100644 examples/src/main/resources/movies/wrote.csv diff --git a/examples/src/main/resources/mini_yelp/business.csv b/examples/src/main/resources/mini_yelp/business.csv deleted file mode 100644 index c4bbc7a086f36..0000000000000 --- a/examples/src/main/resources/mini_yelp/business.csv +++ /dev/null @@ -1,4 +0,0 @@ -id;name;cuisine -3;Mario's Pizza;Italian -4;Bob's Burger;American -5;HermannTheGerman;German diff --git a/examples/src/main/resources/mini_yelp/reviews.csv b/examples/src/main/resources/mini_yelp/reviews.csv deleted file mode 100644 index b2a23d3795a51..0000000000000 --- a/examples/src/main/resources/mini_yelp/reviews.csv +++ /dev/null @@ -1,4 +0,0 @@ -id;user;business;rating -0;0;3;2 -1;1;5;5 -2;0;3;10 diff --git a/examples/src/main/resources/mini_yelp/user.csv b/examples/src/main/resources/mini_yelp/user.csv deleted file mode 100644 index 05882b07aa761..0000000000000 --- a/examples/src/main/resources/mini_yelp/user.csv +++ /dev/null @@ -1,4 +0,0 @@ -id;name -0;Alice -1;Bob -2;Eve 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/CompleteGraphAPIExample.scala b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala index a77a39c998a33..78d46be7b66b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -36,23 +36,25 @@ object CompleteGraphAPIExample { val sc = spark.sparkContext sc.setLogLevel("ERROR") - val csvConfig = Map("header" -> "true", "delimiter" -> ";") + val csvConfig = Map("header" -> "true", "delimiter" -> ",", "inferSchema" -> "true") + val resourcePath = "examples/src/main/resources/movies" // Load node dfs and edge df - val userData = spark.read.options(csvConfig).csv("examples/src/main/resources/mini_yelp/user.csv") - val businessData = spark.read.options(csvConfig).csv("examples/src/main/resources/mini_yelp/business.csv") - val reviewData = spark.read.options(csvConfig).csv("examples/src/main/resources/mini_yelp/reviews.csv") + val moviesData = spark.read.options(csvConfig).csv(s"$resourcePath/movies.csv") + val personsData = spark.read.options(csvConfig).csv(s"$resourcePath/persons.csv") + // TODO: get persons.roles as array type (is not infered correctly) + val actedInData = spark.read.options(csvConfig).csv(s"$resourcePath/acted_in.csv") // Initialise a GraphSession val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val userNodeFrame = NodeFrame.create(userData, "id", Set("User")) - val businessNodeFrame = NodeFrame.create(businessData, "id", Set("Business")) - val reviewRelationshipFrame = RelationshipFrame.create(reviewData, "id", "user", "business", "REVIEWS") + val moviesNodeFrame = NodeFrame.create(moviesData, "id", Set("Movie")) + val personsNodeFrame = NodeFrame.create(personsData, "id", Set("Person")) + val actedInRelationshipFrame = RelationshipFrame.create(actedInData, "id", "source", "target", "ACTED_IN") // Create a PropertyGraph - val graph: PropertyGraph = cypherSession.createGraph(Array(userNodeFrame, businessNodeFrame), Array(reviewRelationshipFrame)) + val graph: PropertyGraph = cypherSession.createGraph(Array(moviesNodeFrame, personsNodeFrame), Array(actedInRelationshipFrame)) // Get existing node labels val labelSet = graph.schema.labels @@ -60,18 +62,17 @@ object CompleteGraphAPIExample { println() StdIn.readLine("Press Enter to continue: ") - val businessNodes = graph.nodeFrame(Array("Business")) + val businessNodes = graph.nodeFrame(Array("Movie")) businessNodes.df.show() StdIn.readLine("Press Enter to continue: ") - // Run parameterized cypher query + // Run parameterised cypher query + val parameters = Map("name" -> "Tom Hanks") val result = graph.cypher( """ - |MATCH (a:User)-[r:REVIEWS]->(b:Business) - |WHERE a.name = $name - |RETURN a, r.rating, b.name""".stripMargin, Map("name" -> "Bob")) - - println("Reviews from Bob") + |MATCH (p:Person {name: $name})-[:ACTED_IN]->(movie) + |RETURN p.name, movie.title""".stripMargin, parameters) + println(s"Movies with ${parameters.get("name")}") result.df.show() // Store the PropertyGraph From 793086b42ffd3fd4d99ba375ba7a46552eabb45f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Florentin=20D=C3=B6rre?= Date: Tue, 8 Oct 2019 14:41:25 +0200 Subject: [PATCH 116/123] Add example with more advanced queries --- .../graph/AdvancedQueriesExample.scala | 113 ++++++++++++++++++ 1 file changed, 113 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala 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..e2ec6c88a499d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -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.examples.graph + +// $example on$ + +import org.apache.spark.cypher.SparkCypherSession +import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} +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") + + // TODO: beautify string column converter & also apply in CompleteGraphAPIExample + // 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 moviesNodeFrame = NodeFrame.create(moviesData, "id", Set("Movie")) + val personsNodeFrame = NodeFrame.create(personsData, "id", Set("Person")) + + val actedInRelationshipFrame = RelationshipFrame.create(actedInData, "id", "source", "target", "ACTED_IN") + val directedRelationshipFrame = RelationshipFrame.create(directedData, "id", "source", "target", "DIRECTED") + val followsRelationshipFrame = RelationshipFrame.create(followsData, "id", "source", "target", "FOLLOWS") + val producedRelationshipFrame = RelationshipFrame.create(producedData, "id", "source", "target", "PRODUCED") + val reviewedRelationshipFrame = RelationshipFrame.create(reviewedData, "id", "source", "target", "REVIEWED") + val wroteRelationshipFrame = RelationshipFrame.create(wroteData, "id", "source", "target", "WROTE") + val relationshipFrames = Array(actedInRelationshipFrame, directedRelationshipFrame, followsRelationshipFrame, + producedRelationshipFrame, reviewedRelationshipFrame, wroteRelationshipFrame) + + // Create a PropertyGraph + val graph: PropertyGraph = cypherSession.createGraph(Array(moviesNodeFrame, personsNodeFrame), relationshipFrames) + + 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 rating + |RETURN title, rating + |ORDER BY rating DESC""".stripMargin) + + println("Best rated movies") + bestMovies.df.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.df.show(false) + StdIn.readLine("Press Enter to continue: ") + + // TODO: Does this query take too long? + val nearKevinBacon = graph.cypher( + """ + |MATCH (bacon:Person {name:"Kevin Bacon"})-[*1..3]-(hollywood) + |RETURN DISTINCT hollywood""".stripMargin) + + println("""Movies and actors up to 2 "hops" away from Kevin Bacon""") + nearKevinBacon.df.show() + + spark.stop() + } +} + From 5f59d28d0df888d4271cb1fc8000356fafb390be Mon Sep 17 00:00:00 2001 From: Paul Horn Date: Fri, 11 Oct 2019 14:57:27 +0200 Subject: [PATCH 117/123] Get examples to work from the IDE MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-Authored-By: Sören Reichardt --- examples/pom.xml | 9 ++- .../graph/AdvancedQueriesExample.scala | 65 +++++++++++++++---- .../graph/CompleteGraphAPIExample.scala | 28 +++++--- .../spark/examples/graph/SimpleExample.scala | 10 +-- 4 files changed, 85 insertions(+), 27 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 22f94c4f76767..5e4838df09712 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -83,6 +83,12 @@ ${project.version} compile + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + compile + org.apache.spark spark-sql_${scala.binary.version} @@ -123,9 +129,6 @@ compile - com.github.scopt - scopt_${scala.binary.version} - 3.7.1 log4j log4j compile 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 index e2ec6c88a499d..b26c7fee69247 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.graph // $example on$ import org.apache.spark.cypher.SparkCypherSession -import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.graph.api.PropertyGraph import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ @@ -32,7 +32,7 @@ object AdvancedQueriesExample { val spark = SparkSession .builder() .appName(s"${this.getClass.getSimpleName}") - .config("spark.master", "local") + .config("spark.master", "local[*]") .getOrCreate() val sc = spark.sparkContext sc.setLogLevel("ERROR") @@ -60,15 +60,55 @@ object AdvancedQueriesExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = NodeFrame.create(moviesData, "id", Set("Movie")) - val personsNodeFrame = NodeFrame.create(personsData, "id", Set("Person")) - - val actedInRelationshipFrame = RelationshipFrame.create(actedInData, "id", "source", "target", "ACTED_IN") - val directedRelationshipFrame = RelationshipFrame.create(directedData, "id", "source", "target", "DIRECTED") - val followsRelationshipFrame = RelationshipFrame.create(followsData, "id", "source", "target", "FOLLOWS") - val producedRelationshipFrame = RelationshipFrame.create(producedData, "id", "source", "target", "PRODUCED") - val reviewedRelationshipFrame = RelationshipFrame.create(reviewedData, "id", "source", "target", "REVIEWED") - val wroteRelationshipFrame = RelationshipFrame.create(wroteData, "id", "source", "target", "WROTE") + val moviesNodeFrame = cypherSession.buildNodeFrame(moviesData) + .idColumn("id") + .labelSet(Array("Movie")) + .properties(Map("title" -> "title")) + .build() + val personsNodeFrame = cypherSession.buildNodeFrame(personsData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + + val actedInRelationshipFrame = cypherSession.buildRelationshipFrame(actedInData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("ACTED_IN") + .properties(Map("roles" -> "roles")) + .build() + val directedRelationshipFrame = cypherSession.buildRelationshipFrame(directedData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("DIRECTED") + .build() + val followsRelationshipFrame = cypherSession.buildRelationshipFrame(followsData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("FOLLOWS") + .build() + val producedRelationshipFrame = cypherSession.buildRelationshipFrame(producedData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("PRODUCED") + .build() + val reviewedRelationshipFrame = cypherSession.buildRelationshipFrame(reviewedData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("REVIEWED") + .properties(Map("rating" -> "rating")) + .build() + val wroteRelationshipFrame = cypherSession.buildRelationshipFrame(wroteData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("WROTE") + .build() val relationshipFrames = Array(actedInRelationshipFrame, directedRelationshipFrame, followsRelationshipFrame, producedRelationshipFrame, reviewedRelationshipFrame, wroteRelationshipFrame) @@ -99,9 +139,10 @@ object AdvancedQueriesExample { StdIn.readLine("Press Enter to continue: ") // TODO: Does this query take too long? + // -- yes :/ val nearKevinBacon = graph.cypher( """ - |MATCH (bacon:Person {name:"Kevin Bacon"})-[*1..3]-(hollywood) + |MATCH (bacon:Person {name:"Kevin Bacon"})-[*1..2]-(hollywood) |RETURN DISTINCT hollywood""".stripMargin) println("""Movies and actors up to 2 "hops" away from Kevin Bacon""") 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 index 78d46be7b66b6..b48f7aa320e19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.graph // $example on$ import org.apache.spark.cypher.SparkCypherSession -import org.apache.spark.graph.api.{NodeFrame, PropertyGraph, RelationshipFrame} +import org.apache.spark.graph.api.PropertyGraph import org.apache.spark.sql.{SaveMode, SparkSession} import scala.io.StdIn @@ -31,7 +31,7 @@ object CompleteGraphAPIExample { val spark = SparkSession .builder() .appName(s"${this.getClass.getSimpleName}") - .config("spark.master", "local") + .config("spark.master", "local[*]") .getOrCreate() val sc = spark.sparkContext sc.setLogLevel("ERROR") @@ -48,9 +48,22 @@ object CompleteGraphAPIExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = NodeFrame.create(moviesData, "id", Set("Movie")) - val personsNodeFrame = NodeFrame.create(personsData, "id", Set("Person")) - val actedInRelationshipFrame = RelationshipFrame.create(actedInData, "id", "source", "target", "ACTED_IN") + val moviesNodeFrame = cypherSession.buildNodeFrame(moviesData) + .idColumn("id") + .labelSet(Array("Movie")) + .properties(Map("title" -> "title")) + .build() + val personsNodeFrame = cypherSession.buildNodeFrame(personsData) + .idColumn("id") + .labelSet(Array("Person")) + .properties(Map("name" -> "name")) + .build() + val actedInRelationshipFrame = cypherSession.buildRelationshipFrame(actedInData) + .idColumn("id") + .sourceIdColumn("source") + .targetIdColumn("target") + .relationshipType("ACTED_IN") + .build() // Create a PropertyGraph @@ -77,11 +90,10 @@ object CompleteGraphAPIExample { // Store the PropertyGraph val savePath = "examples/src/main/resources/exampleGraph/" - graph.save(savePath, SaveMode.Overwrite) + graph.write.mode(SaveMode.Overwrite).save(savePath) // Load the PropertyGraph - val importedGraph = cypherSession.load(savePath) - + 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 index a652fa8ea2903..f4a140891832d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala @@ -19,6 +19,7 @@ package org.apache.spark.examples.graph // $example on$ import org.apache.spark.cypher.SparkCypherSession +import org.apache.spark.graph.api.CypherSession // $example off$ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.SparkSession @@ -29,14 +30,15 @@ object SimpleExample { val spark = SparkSession .builder() .appName(s"${this.getClass.getSimpleName}") + .config("spark.master", "local[*]") .getOrCreate() val sc = spark.sparkContext // Create node df and edge df - val nodeData: DataFrame = spark.createDataFrame(Seq(0 -> "Alice", 1 -> "Bob")) - .toDF("id", "name") - val relationshipData: DataFrame = spark.createDataFrame(Seq((0, 0, 1))) - .toDF("id", "source", "target") + 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) From 31ce35da8f0283d9da9f1f8cd5b3af475591db33 Mon Sep 17 00:00:00 2001 From: Mats Rydberg Date: Tue, 15 Oct 2019 14:42:15 +0200 Subject: [PATCH 118/123] Adapt examples post rebase MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../graph/AdvancedQueriesExample.scala | 22 +++++++++---------- .../graph/CompleteGraphAPIExample.scala | 12 +++++----- .../spark/examples/graph/SimpleExample.scala | 2 +- 3 files changed, 18 insertions(+), 18 deletions(-) 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 index b26c7fee69247..4c7f51547f95e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -60,50 +60,50 @@ object AdvancedQueriesExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = cypherSession.buildNodeFrame(moviesData) + val moviesNodeFrame = cypherSession.buildNodeDataset(moviesData) .idColumn("id") .labelSet(Array("Movie")) .properties(Map("title" -> "title")) .build() - val personsNodeFrame = cypherSession.buildNodeFrame(personsData) + val personsNodeFrame = cypherSession.buildNodeDataset(personsData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val actedInRelationshipFrame = cypherSession.buildRelationshipFrame(actedInData) + val actedInRelationshipFrame = cypherSession.buildRelationshipDataset(actedInData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("ACTED_IN") .properties(Map("roles" -> "roles")) .build() - val directedRelationshipFrame = cypherSession.buildRelationshipFrame(directedData) + val directedRelationshipFrame = cypherSession.buildRelationshipDataset(directedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("DIRECTED") .build() - val followsRelationshipFrame = cypherSession.buildRelationshipFrame(followsData) + val followsRelationshipFrame = cypherSession.buildRelationshipDataset(followsData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("FOLLOWS") .build() - val producedRelationshipFrame = cypherSession.buildRelationshipFrame(producedData) + val producedRelationshipFrame = cypherSession.buildRelationshipDataset(producedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("PRODUCED") .build() - val reviewedRelationshipFrame = cypherSession.buildRelationshipFrame(reviewedData) + val reviewedRelationshipFrame = cypherSession.buildRelationshipDataset(reviewedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("REVIEWED") .properties(Map("rating" -> "rating")) .build() - val wroteRelationshipFrame = cypherSession.buildRelationshipFrame(wroteData) + val wroteRelationshipFrame = cypherSession.buildRelationshipDataset(wroteData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") @@ -123,7 +123,7 @@ object AdvancedQueriesExample { |ORDER BY rating DESC""".stripMargin) println("Best rated movies") - bestMovies.df.show() + bestMovies.ds.show() StdIn.readLine("Press Enter to continue: ") val multiActor = graph.cypher( @@ -135,7 +135,7 @@ object AdvancedQueriesExample { |""".stripMargin) println("Actors playing multiple roles in one movie") - multiActor.df.show(false) + multiActor.ds.show(false) StdIn.readLine("Press Enter to continue: ") // TODO: Does this query take too long? @@ -146,7 +146,7 @@ object AdvancedQueriesExample { |RETURN DISTINCT hollywood""".stripMargin) println("""Movies and actors up to 2 "hops" away from Kevin Bacon""") - nearKevinBacon.df.show() + nearKevinBacon.ds.show() spark.stop() } 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 index b48f7aa320e19..41c4dcfb19e83 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -48,17 +48,17 @@ object CompleteGraphAPIExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = cypherSession.buildNodeFrame(moviesData) + val moviesNodeFrame = cypherSession.buildNodeDataset(moviesData) .idColumn("id") .labelSet(Array("Movie")) .properties(Map("title" -> "title")) .build() - val personsNodeFrame = cypherSession.buildNodeFrame(personsData) + val personsNodeFrame = cypherSession.buildNodeDataset(personsData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val actedInRelationshipFrame = cypherSession.buildRelationshipFrame(actedInData) + val actedInRelationshipFrame = cypherSession.buildRelationshipDataset(actedInData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") @@ -75,8 +75,8 @@ object CompleteGraphAPIExample { println() StdIn.readLine("Press Enter to continue: ") - val businessNodes = graph.nodeFrame(Array("Movie")) - businessNodes.df.show() + val businessNodes = graph.nodeDataset(Array("Movie")) + businessNodes.ds.show() StdIn.readLine("Press Enter to continue: ") // Run parameterised cypher query @@ -86,7 +86,7 @@ object CompleteGraphAPIExample { |MATCH (p:Person {name: $name})-[:ACTED_IN]->(movie) |RETURN p.name, movie.title""".stripMargin, parameters) println(s"Movies with ${parameters.get("name")}") - result.df.show() + result.ds.show() // Store the PropertyGraph val savePath = "examples/src/main/resources/exampleGraph/" 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 index f4a140891832d..4f8674180bf99 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala @@ -53,7 +53,7 @@ object SimpleExample { |RETURN a, r""".stripMargin) // Print the result - result.df.show() + result.ds.show() spark.stop() From 80f9ce60031b3c562f97d877bec7385132064af4 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 15 Oct 2019 17:01:04 +0200 Subject: [PATCH 119/123] Rename and document type column validation --- .../apache/spark/graph/api/CypherSession.scala | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) 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 index 3874cca6e28a3..07a0b3ce7f69e 100644 --- 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 @@ -66,7 +66,7 @@ object CypherSession { */ def extractNodeDatasets(nodes: Dataset[Row]): Array[NodeDataset] = { val labelColumns = nodes.columns.filter(_.startsWith(LABEL_COLUMN_PREFIX)).toSet - validateTypeColumns(nodes.schema, labelColumns, LABEL_COLUMN_PREFIX) + validateLabelOrRelTypeColumns(nodes.schema, labelColumns, LABEL_COLUMN_PREFIX) val nodeProperties = (nodes.columns.toSet - ID_COLUMN -- labelColumns) .map(col => col -> col) @@ -110,7 +110,7 @@ object CypherSession { def extractRelationshipDatasets(relationships: Dataset[Row]): Array[RelationshipDataset] = { val relColumns = relationships.columns.toSet val relTypeColumns = relColumns.filter(_.startsWith(REL_TYPE_COLUMN_PREFIX)) - validateTypeColumns(relationships.schema, relTypeColumns, 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 @@ -127,7 +127,18 @@ object CypherSession { }.toArray } - private def validateTypeColumns( + /** + * 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 = { From cb8ccdcf7840833fe58e95027fd7b7039259be85 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 15 Oct 2019 17:14:28 +0200 Subject: [PATCH 120/123] Inline TODO for relationship validation --- .../main/scala/org/apache/spark/graph/api/CypherSession.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 index 07a0b3ce7f69e..f14a9f216bba0 100644 --- 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 @@ -116,7 +116,8 @@ object CypherSession { 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, From 314ea239ff8290e83bee6367d1f68cba2800cab2 Mon Sep 17 00:00:00 2001 From: Martin Junghanns Date: Tue, 15 Oct 2019 17:35:18 +0200 Subject: [PATCH 121/123] Refactor GraphElementDataset builders to element objects --- .../graph/AdvancedQueriesExample.scala | 18 +++++++++--------- .../graph/CompleteGraphAPIExample.scala | 8 ++++---- .../spark/graph/api/CypherSession.scala | 19 ------------------- .../apache/spark/graph/api/NodeDataset.scala | 13 +++++++++++++ .../spark/graph/api/RelationshipDataset.scala | 14 ++++++++++++++ .../graph/api/JavaPropertyGraphSuite.java | 4 ++-- .../spark/cypher/GraphExamplesSuite.scala | 14 +++++++------- .../spark/cypher/PropertyGraphReadWrite.scala | 4 ++-- .../spark/cypher/PropertyGraphSuite.scala | 14 +++++++------- 9 files changed, 58 insertions(+), 50 deletions(-) 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 index 4c7f51547f95e..19c4652ff4933 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.graph // $example on$ import org.apache.spark.cypher.SparkCypherSession -import org.apache.spark.graph.api.PropertyGraph +import org.apache.spark.graph.api.{NodeDataset, PropertyGraph, RelationshipDataset} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.functions._ @@ -60,50 +60,50 @@ object AdvancedQueriesExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = cypherSession.buildNodeDataset(moviesData) + val moviesNodeFrame = NodeDataset.builder(moviesData) .idColumn("id") .labelSet(Array("Movie")) .properties(Map("title" -> "title")) .build() - val personsNodeFrame = cypherSession.buildNodeDataset(personsData) + val personsNodeFrame = NodeDataset.builder(personsData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val actedInRelationshipFrame = cypherSession.buildRelationshipDataset(actedInData) + val actedInRelationshipFrame = RelationshipDataset.builder(actedInData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("ACTED_IN") .properties(Map("roles" -> "roles")) .build() - val directedRelationshipFrame = cypherSession.buildRelationshipDataset(directedData) + val directedRelationshipFrame = RelationshipDataset.builder(directedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("DIRECTED") .build() - val followsRelationshipFrame = cypherSession.buildRelationshipDataset(followsData) + val followsRelationshipFrame = RelationshipDataset.builder(followsData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("FOLLOWS") .build() - val producedRelationshipFrame = cypherSession.buildRelationshipDataset(producedData) + val producedRelationshipFrame = RelationshipDataset.builder(producedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("PRODUCED") .build() - val reviewedRelationshipFrame = cypherSession.buildRelationshipDataset(reviewedData) + val reviewedRelationshipFrame = RelationshipDataset.builder(reviewedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("REVIEWED") .properties(Map("rating" -> "rating")) .build() - val wroteRelationshipFrame = cypherSession.buildRelationshipDataset(wroteData) + val wroteRelationshipFrame = RelationshipDataset.builder(wroteData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") 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 index 41c4dcfb19e83..768df43c7e3a8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.graph // $example on$ import org.apache.spark.cypher.SparkCypherSession -import org.apache.spark.graph.api.PropertyGraph +import org.apache.spark.graph.api.{NodeDataset, PropertyGraph, RelationshipDataset} import org.apache.spark.sql.{SaveMode, SparkSession} import scala.io.StdIn @@ -48,17 +48,17 @@ object CompleteGraphAPIExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = cypherSession.buildNodeDataset(moviesData) + val moviesNodeFrame = NodeDataset.builder(moviesData) .idColumn("id") .labelSet(Array("Movie")) .properties(Map("title" -> "title")) .build() - val personsNodeFrame = cypherSession.buildNodeDataset(personsData) + val personsNodeFrame = NodeDataset.builder(personsData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val actedInRelationshipFrame = cypherSession.buildRelationshipDataset(actedInData) + val actedInRelationshipFrame = RelationshipDataset.builder(actedInData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") 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 index f14a9f216bba0..5e34e61aded88 100644 --- 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 @@ -288,23 +288,4 @@ trait CypherSession { */ def read: PropertyGraphReader - /** - * 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 buildNodeDataset(ds: Dataset[Row]): NodeDatasetBuilder = - new NodeDatasetBuilder(ds) - - /** - * 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 buildRelationshipDataset(ds: Dataset[Row]): RelationshipDatasetBuilder = - new RelationshipDatasetBuilder(ds) - } 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 index 08df9aa182273..51a07d0eb6b34 100644 --- 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 @@ -20,6 +20,19 @@ 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. * 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 index 711acbc17a36b..ab4bb2b91e6f3 100644 --- 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 @@ -20,6 +20,20 @@ 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. * 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 index df986a516eaf4..a1f1b4b03140e 100644 --- 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 @@ -80,14 +80,14 @@ public void testCreateFromNodeDataset() { List knowsData = Collections.singletonList(RowFactory.create(0L, 0L, 1L, 1984)); Dataset personDf = spark.createDataFrame(personData, personSchema); - NodeDataset personNodeDataset = cypherSession.buildNodeDataset(personDf) + 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 = cypherSession.buildRelationshipDataset(knowsDf) + RelationshipDataset knowsRelDataset = RelationshipDataset.builder(knowsDf) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") 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 index a4ac474c10f31..ed13540c0156d 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/GraphExamplesSuite.scala @@ -26,7 +26,7 @@ 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 = cypherSession.buildNodeDataset(nodeData) + val nodeDataset: NodeDataset = NodeDataset.builder(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) @@ -39,12 +39,12 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { 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 = cypherSession.buildNodeDataset(nodeData) + val nodeDataset: NodeDataset = NodeDataset.builder(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val relationshipDataset: RelationshipDataset = cypherSession.buildRelationshipDataset(relationshipData) + val relationshipDataset: RelationshipDataset = RelationshipDataset.builder(relationshipData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") @@ -62,13 +62,13 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { 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 = cypherSession.buildNodeDataset(studentDS) + val studentNF = NodeDataset.builder(studentDS) .idColumn("id") .labelSet(Array("Person", "Student")) .properties(Map("name" -> "name", "age" -> "age")) .build() - val teacherNF = cypherSession.buildNodeDataset(teacherDS) + val teacherNF = NodeDataset.builder(teacherDS) .idColumn("id") .labelSet(Array("Person", "Teacher")) .properties(Map("name" -> "name", "subject" -> "subject")) @@ -77,14 +77,14 @@ class GraphExamplesSuite extends SparkFunSuite with SharedCypherContext { 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 = cypherSession.buildRelationshipDataset(knowsDF) + val knowsRF: RelationshipDataset = RelationshipDataset.builder(knowsDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("KNOWS") .properties(Map("since" -> "since")) .build() - val teachesRF: RelationshipDataset = cypherSession.buildRelationshipDataset(teachesDF) + val teachesRF: RelationshipDataset = RelationshipDataset.builder(teachesDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") 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 index 72eb34d4ed022..702ea28c9b116 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphReadWrite.scala @@ -51,7 +51,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef Tuple3(0, 0, 1) )).toDF("id", "source", "target") - private lazy val nodeDataFrame: NodeDataset = cypherSession.buildNodeDataset(nodeData) + private lazy val nodeDataFrame: NodeDataset = NodeDataset.builder(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) @@ -71,7 +71,7 @@ class PropertyGraphReadWrite extends QueryTest with SharedCypherContext with Bef test("save and loads a property graph") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeFrame = cypherSession.buildNodeDataset(nodeData) + val nodeFrame = NodeDataset.builder(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) 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 index 9f764c33fb9bf..1abe0ee6aed39 100644 --- a/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala +++ b/graph/cypher/src/test/scala/org/apache/spark/cypher/PropertyGraphSuite.scala @@ -76,7 +76,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeDataset") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeDataset = cypherSession.buildNodeDataset(nodeData) + val nodeDataset = NodeDataset.builder(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) @@ -92,7 +92,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers test("create graph from NodeDataset and RelationshipDataset") { val nodeData = spark.createDataFrame(Seq(0L -> "Alice", 1L -> "Bob")).toDF("id", "name") - val nodeDataset = cypherSession.buildNodeDataset(nodeData) + val nodeDataset = NodeDataset.builder(nodeData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) @@ -100,7 +100,7 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers val relationshipData = spark .createDataFrame(Seq((0L, 0L, 1L, 1984))) .toDF("id", "source", "target", "since") - val relationshipDataset = cypherSession.buildRelationshipDataset(relationshipData) + val relationshipDataset = RelationshipDataset.builder(relationshipData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") @@ -130,13 +130,13 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((2L, "Eve", "CS"))) .toDF("id", "name", "subject") - val studentNodeDataset = cypherSession.buildNodeDataset(studentDF) + val studentNodeDataset = NodeDataset.builder(studentDF) .idColumn("id") .labelSet(Array("Person", "Student")) .properties(Map("name" -> "name", "age" -> "age")) .build() - val teacherNodeDataset = cypherSession.buildNodeDataset(teacherDF) + val teacherNodeDataset = NodeDataset.builder(teacherDF) .idColumn("id") .labelSet(Array("Person", "Teacher")) .properties(Map("name" -> "name", "subject" -> "subject")) @@ -149,14 +149,14 @@ class PropertyGraphSuite extends QueryTest with SharedSparkSession with Matchers .createDataFrame(Seq((1L, 2L, 1L))) .toDF("id", "source", "target") - val knowsRelationshipDataset = cypherSession.buildRelationshipDataset(knowsDF) + val knowsRelationshipDataset = RelationshipDataset.builder(knowsDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("KNOWS") .properties(Map("since" -> "since")) .build() - val teachesRelationshipDataset = cypherSession.buildRelationshipDataset(teachesDF) + val teachesRelationshipDataset = RelationshipDataset.builder(teachesDF) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") From 3420b6c881d2c1cfb71ae5a5aaffecff08bb18dc Mon Sep 17 00:00:00 2001 From: Mats Rydberg Date: Wed, 16 Oct 2019 11:13:07 +0200 Subject: [PATCH 122/123] Beautify example code MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../graph/AdvancedQueriesExample.scala | 26 +++++++++---------- .../graph/CompleteGraphAPIExample.scala | 17 +++++++++--- .../spark/examples/graph/SimpleExample.scala | 17 ++++++------ .../spark/graph/api/CypherSession.scala | 4 ++- 4 files changed, 38 insertions(+), 26 deletions(-) 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 index 19c4652ff4933..2600364538118 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -37,7 +37,6 @@ object AdvancedQueriesExample { val sc = spark.sparkContext sc.setLogLevel("ERROR") - val csvConfig = Map("header" -> "true", "delimiter" -> ",", "inferSchema" -> "true") val resourcePath = "examples/src/main/resources/movies" @@ -45,7 +44,6 @@ object AdvancedQueriesExample { val moviesData = spark.read.options(csvConfig).csv(s"$resourcePath/movies.csv") val personsData = spark.read.options(csvConfig).csv(s"$resourcePath/persons.csv") - // TODO: beautify string column converter & also apply in CompleteGraphAPIExample // Load edge dfs val actedInDF = spark.read.options(csvConfig).csv(s"$resourcePath/acted_in.csv") val stringToArrayColumn = split(regexp_replace(actedInDF.col("roles"), "\\[|\\]|\"", " "), ",") @@ -60,60 +58,62 @@ object AdvancedQueriesExample { val cypherSession = SparkCypherSession.create(spark) // Create Node- and RelationshipFrames - val moviesNodeFrame = NodeDataset.builder(moviesData) + val movieNodes: NodeDataset = NodeDataset.builder(moviesData) .idColumn("id") .labelSet(Array("Movie")) .properties(Map("title" -> "title")) .build() - val personsNodeFrame = NodeDataset.builder(personsData) + val personNodes: NodeDataset = NodeDataset.builder(personsData) .idColumn("id") .labelSet(Array("Person")) .properties(Map("name" -> "name")) .build() - val actedInRelationshipFrame = RelationshipDataset.builder(actedInData) + val actedInRelationships: RelationshipDataset = RelationshipDataset.builder(actedInData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("ACTED_IN") .properties(Map("roles" -> "roles")) .build() - val directedRelationshipFrame = RelationshipDataset.builder(directedData) + val directedRelationships: RelationshipDataset = RelationshipDataset.builder(directedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("DIRECTED") .build() - val followsRelationshipFrame = RelationshipDataset.builder(followsData) + val followsRelationships: RelationshipDataset = RelationshipDataset.builder(followsData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("FOLLOWS") .build() - val producedRelationshipFrame = RelationshipDataset.builder(producedData) + val producedRelationships: RelationshipDataset = RelationshipDataset.builder(producedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("PRODUCED") .build() - val reviewedRelationshipFrame = RelationshipDataset.builder(reviewedData) + val reviewedRelationships: RelationshipDataset = RelationshipDataset.builder(reviewedData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("REVIEWED") .properties(Map("rating" -> "rating")) .build() - val wroteRelationshipFrame = RelationshipDataset.builder(wroteData) + val wroteRelationships: RelationshipDataset = RelationshipDataset.builder(wroteData) .idColumn("id") .sourceIdColumn("source") .targetIdColumn("target") .relationshipType("WROTE") .build() - val relationshipFrames = Array(actedInRelationshipFrame, directedRelationshipFrame, followsRelationshipFrame, - producedRelationshipFrame, reviewedRelationshipFrame, wroteRelationshipFrame) + val relationships = Array( + actedInRelationships, directedRelationships, followsRelationships, + producedRelationships, reviewedRelationships, wroteRelationships + ) // Create a PropertyGraph - val graph: PropertyGraph = cypherSession.createGraph(Array(moviesNodeFrame, personsNodeFrame), relationshipFrames) + val graph: PropertyGraph = cypherSession.createGraph(Array(movieNodes, personNodes), relationships) val bestMovies = graph.cypher( """ 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 index 768df43c7e3a8..da9889f13774e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/CompleteGraphAPIExample.scala @@ -41,9 +41,15 @@ object CompleteGraphAPIExample { // 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") - // TODO: get persons.roles as array type (is not infered correctly) 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) @@ -65,9 +71,11 @@ object CompleteGraphAPIExample { .relationshipType("ACTED_IN") .build() - // Create a PropertyGraph - val graph: PropertyGraph = cypherSession.createGraph(Array(moviesNodeFrame, personsNodeFrame), Array(actedInRelationshipFrame)) + val graph: PropertyGraph = cypherSession.createGraph( + Array(moviesNodeFrame, personsNodeFrame), + Array(actedInRelationshipFrame) + ) // Get existing node labels val labelSet = graph.schema.labels @@ -84,7 +92,8 @@ object CompleteGraphAPIExample { val result = graph.cypher( """ |MATCH (p:Person {name: $name})-[:ACTED_IN]->(movie) - |RETURN p.name, movie.title""".stripMargin, parameters) + |RETURN p.name, movie.title + """.stripMargin, parameters) println(s"Movies with ${parameters.get("name")}") result.ds.show() 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 index 4f8674180bf99..631fe630e6e9d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/SimpleExample.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.graph // $example on$ import org.apache.spark.cypher.SparkCypherSession -import org.apache.spark.graph.api.CypherSession +import org.apache.spark.graph.api.{CypherResult, CypherSession, PropertyGraph} // $example off$ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.SparkSession @@ -32,30 +32,31 @@ object SimpleExample { .appName(s"${this.getClass.getSimpleName}") .config("spark.master", "local[*]") .getOrCreate() - val sc = spark.sparkContext // Create node df and edge df - val nodeData: DataFrame = spark.createDataFrame(Seq((0, "Alice", true), (1, "Bob", true))) + 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))) + 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 = cypherSession.createGraph(nodeData, relationshipData) + val graph: PropertyGraph = cypherSession.createGraph(nodeData, relationshipData) // Run our first query - val result = graph.cypher( + val result: CypherResult = graph.cypher( """ |MATCH (a:Person)-[r:KNOWS]->(:Person) - |RETURN a, r""".stripMargin) + |RETURN a, r + """.stripMargin) // Print the result result.ds.show() - spark.stop() } } 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 index 5e34e61aded88..c5bacb365107c 100644 --- 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 @@ -150,7 +150,9 @@ object CypherSession { }) columns.foreach(typeColumn => { if (typeColumn.sliding(prefix.length).count(_ == prefix) != 1) { - throw new IllegalArgumentException(s"Type column $typeColumn must contain exactly one type.") + throw new IllegalArgumentException( + s"Type column $typeColumn must contain exactly one type." + ) } }) } From 3786c73442cfb5c0b1e2c475c8259cd99ac7a8a3 Mon Sep 17 00:00:00 2001 From: Mats Rydberg Date: Thu, 17 Oct 2019 10:12:51 +0200 Subject: [PATCH 123/123] Beautify example code MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: Max Kießling --- .../graph/AdvancedQueriesExample.scala | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) 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 index 2600364538118..fbaf156bcaaa2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graph/AdvancedQueriesExample.scala @@ -19,7 +19,8 @@ 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.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._ @@ -114,13 +115,15 @@ object AdvancedQueriesExample { // 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 rating - |RETURN title, rating - |ORDER BY rating DESC""".stripMargin) + |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() @@ -130,7 +133,7 @@ object AdvancedQueriesExample { """ |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 + |RETURN p.name AS actor, m.title AS movie, r.roles AS roles |ORDER BY size(roles) DESC |""".stripMargin) @@ -138,17 +141,20 @@ object AdvancedQueriesExample { multiActor.ds.show(false) StdIn.readLine("Press Enter to continue: ") - // TODO: Does this query take too long? - // -- yes :/ val nearKevinBacon = graph.cypher( """ - |MATCH (bacon:Person {name:"Kevin Bacon"})-[*1..2]-(hollywood) - |RETURN DISTINCT hollywood""".stripMargin) + |MATCH (bacon:Person {name: 'Kevin Bacon'})-[*1..3]-(hollywood) + |RETURN DISTINCT hollywood + """.stripMargin) - println("""Movies and actors up to 2 "hops" away from Kevin Bacon""") + 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) + } }