From ed591b88f22b92b0bc3da73d928cab5167ade16c Mon Sep 17 00:00:00 2001 From: Derrick Oswald Date: Sat, 21 Jul 2018 15:24:00 +0200 Subject: [PATCH] update to Spark 2.2.2 --- pom.xml | 11 ++++-- .../scala/ch/ninecode/cim/CIMEdgeData.scala | 1 + .../cim/CIMNetworkTopologyProcessor.scala | 12 +++---- .../ch/ninecode/cim/CIMTopologyOptions.scala | 35 +++++++++++++++++++ .../scala/ch/ninecode/cim/CIMVertexData.scala | 1 + 5 files changed, 52 insertions(+), 8 deletions(-) create mode 100644 src/main/scala/ch/ninecode/cim/CIMTopologyOptions.scala diff --git a/pom.xml b/pom.xml index d7df6fcc7..8ec9c391d 100644 --- a/pom.xml +++ b/pom.xml @@ -4,7 +4,7 @@ ch.ninecode.cim CIMReader - 2.11-2.2.1-2.8.0 + 2.11-2.2.2-2.9.0 ${project.artifactId} Expose CIM data files as Spark RDD 2015 @@ -69,12 +69,19 @@ + UTF-8 1.8 1.8 + + 2.11 2.11.8 - 2.2.1 + + + 2.2.2 + + 3.6.0 4.12 3.0.3 diff --git a/src/main/scala/ch/ninecode/cim/CIMEdgeData.scala b/src/main/scala/ch/ninecode/cim/CIMEdgeData.scala index e568df72e..5d382b756 100644 --- a/src/main/scala/ch/ninecode/cim/CIMEdgeData.scala +++ b/src/main/scala/ch/ninecode/cim/CIMEdgeData.scala @@ -8,6 +8,7 @@ package ch.ninecode.cim * @param id_seq_2 the mRID of terminal 1 (or N in the case of multi-terminal devices) * @param id_cn_2 the connectivity node of terminal 1 (or N in the case of multi-terminal devices) * @param id_equ the [[ch.ninecode.model.ConductingEquipment]] object associated with the terminals + * @param voltage the nominal voltage of the edge * @param isZero true if there is no electrical difference between the terminals, i.e. a closed switch, * which means the terminals are the same topological node * @param isConnected true if there is a connection between the terminals, i.e. a cable, diff --git a/src/main/scala/ch/ninecode/cim/CIMNetworkTopologyProcessor.scala b/src/main/scala/ch/ninecode/cim/CIMNetworkTopologyProcessor.scala index 37e9cdf4a..f5fc41aa8 100644 --- a/src/main/scala/ch/ninecode/cim/CIMNetworkTopologyProcessor.scala +++ b/src/main/scala/ch/ninecode/cim/CIMNetworkTopologyProcessor.scala @@ -575,9 +575,9 @@ with // get the edges that have different topological nodes on each end val nodes = graph.vertices.values.keyBy (_.node).distinct // distinct topological nodes val pairs = graph.vertices.keyBy (_._2.node).join (nodes).map ( x => (x._2._1._1, x._2._2)) // get vertex-node pairs - val b1 = graph.edges.keyBy ((edge) => edge.dstId).join (pairs) // match edge end 1 - val b2 = b1.values.keyBy ((edge) => edge._1.srcId).join (pairs) // match edge end 2 - val b3 = b2.values.map ((edge) => (edge._1._1, edge._1._2, edge._2)) // simplify + val b1 = graph.edges.keyBy (edge => edge.dstId).join (pairs) // match edge end 1 + val b2 = b1.values.keyBy (edge => edge._1.srcId).join (pairs) // match edge end 2 + val b3 = b2.values.map (edge => (edge._1._1, edge._1._2, edge._2)) // simplify val boundaries = b3.filter (boundary) // keep edges with different nodes on each end // construct the topological graph from the edges @@ -637,7 +637,7 @@ with val elements = get[Element]("Elements").keyBy (_.id) val terms = get[Terminal].keyBy (_.ConductingEquipment).join (elements).values // map each graph vertex to the terminals - val vertices = get[ConnectivityNode].map ((x) => (vertex_id (x.id), x)) + val vertices = get[ConnectivityNode].map (x => (vertex_id (x.id), x)) val td_plus = graph.vertices.join (vertices).values.filter (_._1.island != 0L).keyBy (_._2.id).leftOuterJoin (terms.keyBy (_._1.ConnectivityNode)).values val islands = td_plus.groupBy (_._1._1.island).values.map (to_islands) @@ -655,14 +655,14 @@ with TopologicalIsland.subsetter.save (session.sqlContext, new_ti.asInstanceOf[TopologicalIsland.subsetter.rddtype], storage) val nodes_with_islands = graph.vertices.values.keyBy (_.island).join (islands).values - val nodes = nodes_with_islands.groupBy (_._1.node).map ((x) => (x._1, x._2.head._1, Some (x._2.head._2))).map (to_nodes) + val nodes = nodes_with_islands.groupBy (_._1.node).map (x => (x._1, x._2.head._1, Some (x._2.head._2))).map (to_nodes) if (debug && log.isDebugEnabled) log.debug (nodes.count + " nodes") (nodes, new_ti) } else { - val nodes = graph.vertices.values.groupBy (_.node).map ((x) => (x._1, x._2.head, None)).map (to_nodes) + val nodes = graph.vertices.values.groupBy (_.node).map (x => (x._1, x._2.head, None)).map (to_nodes) if (debug && log.isDebugEnabled) log.debug (nodes.count + " nodes") (nodes, spark.sparkContext.emptyRDD[TopologicalIsland]) diff --git a/src/main/scala/ch/ninecode/cim/CIMTopologyOptions.scala b/src/main/scala/ch/ninecode/cim/CIMTopologyOptions.scala new file mode 100644 index 000000000..a45664f14 --- /dev/null +++ b/src/main/scala/ch/ninecode/cim/CIMTopologyOptions.scala @@ -0,0 +1,35 @@ +package ch.ninecode.cim + +trait State + +case class ForceTrue () extends State +case class ForceFalse () extends State +case class Unforced () extends State + +/** + * Topological processing options. + * + * This class determines some of the behaviour of the CIMNetworkTopologyProcessor. + * These options are passed to the processor constructor and cannot be altered dynamically. + * + * @param force_retain_switches Allows override of the behaviour when the processor encounters a Switch + * or a Switch derived class (e.g. Breaker) except for Fuse. The default behaviour of Unforced + * will use the value of the retained attribute to identify an island boundary + * only if the attribute is present in the CIM file and the value is true. + * When set to ForceTrue the behaviour is equivalent to having a retained + * attribute with value true for every instance. + * When set to ForecFalse the behaviour is equivalent to having a retained + * attribute with value false for every instance. + * @param force_retain_fuses Allows override of the behaviour when a Fuse is encountered. + * The same effect as for Switch objects using force_retain_switches but for Fuse objects. + * @param default_switch_closed_state Allows changing the behaviour when the processor encounters a Switch + * that has neither an open attribute, nor normalOpen attribute. + * The default behaviour of ForceTrue is the same as if open and normalOpen + * both specify false. + */ +case class CIMTopologyOptions +( + force_retain_switches: State = Unforced (), + force_retain_fuses: State = Unforced (), + default_switch_closed_state: State = ForceTrue () +) \ No newline at end of file diff --git a/src/main/scala/ch/ninecode/cim/CIMVertexData.scala b/src/main/scala/ch/ninecode/cim/CIMVertexData.scala index d74a0e0cf..7b76c61eb 100644 --- a/src/main/scala/ch/ninecode/cim/CIMVertexData.scala +++ b/src/main/scala/ch/ninecode/cim/CIMVertexData.scala @@ -9,6 +9,7 @@ import org.apache.spark.graphx.VertexId * @param island_label a user friendly label for the island * @param node the minimum (hash code) of equivalent ConnectivityNode (a single topological node) * @param node_label a user friendly label for the node + * @param voltage the nominal voltage of the node */ case class CIMVertexData (var island: VertexId = Long.MaxValue, var island_label: String = "", var node: VertexId = Long.MaxValue, var node_label: String = "", var voltage: String = null) extends Serializable {