001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018 019package org.apache.oozie.util; 020 021import edu.uci.ics.jung.algorithms.layout.StaticLayout; 022import edu.uci.ics.jung.graph.DirectedSparseGraph; 023import edu.uci.ics.jung.graph.Graph; 024import edu.uci.ics.jung.graph.util.Context; 025import edu.uci.ics.jung.visualization.VisualizationImageServer; 026import edu.uci.ics.jung.visualization.renderers.Renderer; 027import edu.uci.ics.jung.visualization.util.ArrowFactory; 028import org.apache.commons.collections15.Transformer; 029import org.apache.oozie.WorkflowJobBean; 030import org.apache.oozie.client.WorkflowAction; 031import org.apache.oozie.client.WorkflowAction.Status; 032import org.apache.oozie.client.WorkflowJob; 033import org.xml.sax.Attributes; 034import org.xml.sax.InputSource; 035import org.xml.sax.SAXException; 036import org.xml.sax.XMLReader; 037import org.xml.sax.helpers.DefaultHandler; 038 039import javax.imageio.ImageIO; 040import javax.xml.parsers.SAXParser; 041import javax.xml.parsers.SAXParserFactory; 042import java.awt.*; 043import java.awt.geom.Ellipse2D; 044import java.awt.geom.Point2D; 045import java.awt.image.BufferedImage; 046import java.io.IOException; 047import java.io.OutputStream; 048import java.io.StringReader; 049import java.util.HashMap; 050import java.util.Iterator; 051import java.util.LinkedHashMap; 052import java.util.Map; 053 054/** 055 * Class to generate and plot runtime workflow DAG 056 */ 057public class GraphGenerator { 058 059 private String xml; 060 private WorkflowJobBean job; 061 private boolean showKill = false; 062 private final int actionsLimit = 25; 063 064 /** 065 * C'tor 066 * @param xml The workflow definition XML 067 * @param job Current status of the job 068 * @param showKill Flag to whether show 'kill' node 069 */ 070 public GraphGenerator(String xml, WorkflowJobBean job, boolean showKill) { 071 if(job == null) { 072 throw new IllegalArgumentException("JsonWorkflowJob can't be null"); 073 } 074 this.xml = xml; 075 this.job = job; 076 this.showKill = showKill; 077 } 078 079 /** 080 * C'tor 081 * @param xml 082 * @param job 083 */ 084 public GraphGenerator(String xml, WorkflowJobBean job) { 085 this(xml, job, false); 086 } 087 088 /** 089 * Overridden to thwart finalizer attack 090 */ 091 @Override 092 public final void finalize() { 093 // No-op; just to avoid finalizer attack 094 // as the constructor is throwing an exception 095 } 096 097 /** 098 * Stream the PNG file to client 099 * @param out 100 * @throws Exception 101 */ 102 public void write(OutputStream out) throws Exception { 103 SAXParserFactory spf = SAXParserFactory.newInstance(); 104 spf.setFeature("http://xml.org/sax/features/external-general-entities", false); 105 spf.setFeature("http://xml.org/sax/features/external-parameter-entities", false); 106 spf.setFeature("http://apache.org/xml/features/disallow-doctype-decl", true); 107 spf.setNamespaceAware(true); 108 SAXParser saxParser = spf.newSAXParser(); 109 XMLReader xmlReader = saxParser.getXMLReader(); 110 xmlReader.setContentHandler(new XMLParser(out)); 111 xmlReader.parse(new InputSource(new StringReader(xml))); 112 } 113 114 private class XMLParser extends DefaultHandler { 115 116 private OutputStream out; 117 private LinkedHashMap<String, OozieWFNode> tags; 118 119 private String action = null; 120 private String actionOK = null; 121 private String actionErr = null; 122 private String actionType = null; 123 private String fork; 124 private String decision; 125 126 public XMLParser(OutputStream out) { 127 this.out = out; 128 } 129 130 @Override 131 public void startDocument() throws SAXException { 132 tags = new LinkedHashMap(); 133 } 134 135 @Override 136 public void endDocument() throws SAXException { 137 138 if(tags.isEmpty()) { 139 // Nothing to do here! 140 return; 141 } 142 143 int maxX = Integer.MIN_VALUE; 144 int maxY = Integer.MIN_VALUE; 145 int minX = Integer.MAX_VALUE; 146 int currX = 45; 147 int currY = 45; 148 final int xMargin = 205; 149 final int yMargin = 50; 150 final int xIncr = 215; // The widest element is 200 pixels (Rectangle) 151 final int yIncr = 255; // The tallest element is 150 pixels; (Diamond) 152 HashMap<String, WorkflowAction> actionMap = new HashMap<String, WorkflowAction>(); 153 154 // Create a hashmap for faster lookups 155 // Also override showKill if there's any failed action 156 boolean found = false; 157 for(WorkflowAction wfAction : job.getActions()) { 158 actionMap.put(wfAction.getName(), wfAction); 159 if(!found) { 160 switch(wfAction.getStatus()) { 161 case KILLED: 162 case ERROR: 163 case FAILED: 164 showKill = true; // Assuming on error the workflow eventually ends with kill node 165 found = true; 166 } 167 } 168 } 169 170 // Start building the graph 171 DirectedSparseGraph<OozieWFNode, String> dg = new DirectedSparseGraph<OozieWFNode, String>(); 172 for(Map.Entry<String, OozieWFNode> entry : tags.entrySet()) { 173 String name = entry.getKey(); 174 OozieWFNode node = entry.getValue(); 175 if(actionMap.containsKey(name)) { 176 node.setStatus(actionMap.get(name).getStatus()); 177 } 178 179 // Set (x,y) coords of the vertices if not already set 180 if(node.getLocation().equals(new Point(0, 0))) { 181 node.setLocation(currX, currY); 182 } 183 184 float childStep = showKill ? -(((float)node.getArcs().size() - 1 ) / 2) 185 : -((float)node.getArcs().size() / 2 - 1); 186 int nodeX = node.getLocation().x; 187 int nodeY = node.getLocation().y; 188 for(Map.Entry<String, Boolean> arc : node.getArcs().entrySet()) { 189 if(!showKill && arc.getValue() && tags.get(arc.getKey()).getType().equals("kill")) { 190 // Don't show kill node (assumption: only error goes to kill node; 191 // No ok goes to kill node) 192 continue; 193 } 194 OozieWFNode child = tags.get(arc.getKey()); 195 if(child == null) { 196 continue; // or throw error? 197 } 198 dg.addEdge(name + "-->" + arc.getKey(), node, child); 199 // TODO: Experimental -- should we set coords even if they're already set? 200 //if(child.getLocation().equals(new Point(0, 0))) { 201 int childX = (int)(nodeX + childStep * xIncr); 202 int childY = nodeY + yIncr; 203 child.setLocation(childX, childY); 204 205 if(minX > childX) { 206 minX = childX; 207 } 208 if(maxX < childX) { 209 maxX = childX; 210 } 211 if(maxY < childY) { 212 maxY = childY; 213 } 214 //} 215 childStep += 1; 216 } 217 218 currY += yIncr; 219 currX = nodeX; 220 if(minX > nodeX) { 221 minX = nodeX; 222 } 223 if(maxX < nodeX) { 224 maxX = nodeX; 225 } 226 if(maxY < nodeY) { 227 maxY = nodeY; 228 } 229 } // Done building graph 230 231 final int padX = minX < 0 ? -minX: 0; 232 233 Transformer<OozieWFNode, Point2D> locationInit = new Transformer<OozieWFNode, Point2D>() { 234 235 @Override 236 public Point2D transform(OozieWFNode node) { 237 if(padX == 0) { 238 return node.getLocation(); 239 } else { 240 return new Point(node.getLocation().x + padX + xMargin, node.getLocation().y); 241 } 242 } 243 244 }; 245 246 StaticLayout<OozieWFNode, String> layout = new StaticLayout<OozieWFNode, String>(dg, locationInit, new Dimension(maxX + padX + xMargin, maxY)); 247 layout.lock(true); 248 VisualizationImageServer<OozieWFNode, String> vis = new VisualizationImageServer<OozieWFNode, String>(layout, new Dimension(maxX + padX + 2 * xMargin, maxY + yMargin)); 249 250 vis.getRenderContext().setEdgeArrowTransformer(new ArrowShapeTransformer()); 251 vis.getRenderContext().setArrowDrawPaintTransformer(new ArcPaintTransformer()); 252 vis.getRenderContext().setEdgeDrawPaintTransformer(new ArcPaintTransformer()); 253 vis.getRenderContext().setEdgeStrokeTransformer(new ArcStrokeTransformer()); 254 vis.getRenderContext().setVertexShapeTransformer(new NodeShapeTransformer()); 255 vis.getRenderContext().setVertexFillPaintTransformer(new NodePaintTransformer()); 256 vis.getRenderContext().setVertexStrokeTransformer(new NodeStrokeTransformer()); 257 vis.getRenderContext().setVertexLabelTransformer(new NodeLabelTransformer()); 258 vis.getRenderContext().setVertexFontTransformer(new NodeFontTransformer()); 259 vis.getRenderer().getVertexLabelRenderer().setPosition(Renderer.VertexLabel.Position.CNTR); 260 vis.setBackground(Color.WHITE); 261 262 Dimension d = vis.getSize(); 263 BufferedImage img = new BufferedImage(d.width, d.height, BufferedImage.TYPE_INT_RGB); 264 Graphics2D g = img.createGraphics(); 265 vis.paintAll(g); 266 267 try { 268 ImageIO.write(img, "png", out); 269 } 270 catch (IOException ioe) { 271 throw new SAXException(ioe); 272 } 273 finally { 274 try { 275 out.close(); //closing connection is imperative 276 //regardless of ImageIO.write throwing exception or not 277 //hence in finally block 278 } 279 catch (IOException e) { 280 XLog.getLog(getClass()).trace("Exception while closing OutputStream"); 281 } 282 out = null; 283 img.flush(); 284 g.dispose(); 285 vis.removeAll(); 286 } 287 } 288 289 @Override 290 public void startElement(String namespaceURI, 291 String localName, 292 String qName, 293 Attributes atts) 294 throws SAXException { 295 if(localName.equalsIgnoreCase("start")) { 296 String start = localName.toLowerCase(); 297 if(!tags.containsKey(start)) { 298 OozieWFNode v = new OozieWFNode(start, start); 299 v.addArc(atts.getValue("to")); 300 tags.put(start, v); 301 } 302 } else if(localName.equalsIgnoreCase("action")) { 303 action = atts.getValue("name"); 304 } else if(action != null && actionType == null) { 305 actionType = localName.toLowerCase(); 306 } else if(localName.equalsIgnoreCase("ok") && action != null && actionOK == null) { 307 actionOK = atts.getValue("to"); 308 } else if(localName.equalsIgnoreCase("error") && action != null && actionErr == null) { 309 actionErr = atts.getValue("to"); 310 } else if(localName.equalsIgnoreCase("fork")) { 311 fork = atts.getValue("name"); 312 if(!tags.containsKey(fork)) { 313 tags.put(fork, new OozieWFNode(fork, localName.toLowerCase())); 314 } 315 } else if(localName.equalsIgnoreCase("path")) { 316 tags.get(fork).addArc(atts.getValue("start")); 317 } else if(localName.equalsIgnoreCase("join")) { 318 String join = atts.getValue("name"); 319 if(!tags.containsKey(join)) { 320 OozieWFNode v = new OozieWFNode(join, localName.toLowerCase()); 321 v.addArc(atts.getValue("to")); 322 tags.put(join, v); 323 } 324 } else if(localName.equalsIgnoreCase("decision")) { 325 decision = atts.getValue("name"); 326 if(!tags.containsKey(decision)) { 327 tags.put(decision, new OozieWFNode(decision, localName.toLowerCase())); 328 } 329 } else if(localName.equalsIgnoreCase("case") 330 || localName.equalsIgnoreCase("default")) { 331 tags.get(decision).addArc(atts.getValue("to")); 332 } else if(localName.equalsIgnoreCase("kill") 333 || localName.equalsIgnoreCase("end")) { 334 String name = atts.getValue("name"); 335 if(!tags.containsKey(name)) { 336 tags.put(name, new OozieWFNode(name, localName.toLowerCase())); 337 } 338 } 339 if (tags.size() > actionsLimit) { 340 tags.clear(); 341 throw new SAXException("Can't display the graph. Number of actions are more than display limit " + actionsLimit); 342 } 343 } 344 345 @Override 346 public void endElement(String namespaceURI, 347 String localName, 348 String qName) 349 throws SAXException { 350 if(localName.equalsIgnoreCase("action")) { 351 tags.put(action, new OozieWFNode(action, actionType)); 352 tags.get(action).addArc(this.actionOK); 353 tags.get(action).addArc(this.actionErr, true); 354 action = null; 355 actionOK = null; 356 actionErr = null; 357 actionType = null; 358 } 359 } 360 361 private class OozieWFNode { 362 private String name; 363 private String type; 364 private Point loc; 365 private HashMap<String, Boolean> arcs; 366 private Status status = null; 367 368 public OozieWFNode(String name, 369 String type, 370 HashMap<String, Boolean> arcs, 371 Point loc, 372 Status status) { 373 this.name = name; 374 this.type = type; 375 this.arcs = arcs; 376 this.loc = loc; 377 this.status = status; 378 } 379 380 public OozieWFNode(String name, String type, HashMap<String, Boolean> arcs) { 381 this(name, type, arcs, new Point(0, 0), null); 382 } 383 384 public OozieWFNode(String name, String type) { 385 this(name, type, new HashMap<String, Boolean>(), new Point(0, 0), null); 386 } 387 388 public OozieWFNode(String name, String type, WorkflowAction.Status status) { 389 this(name, type, new HashMap<String, Boolean>(), new Point(0, 0), status); 390 } 391 392 public void addArc(String arc, boolean isError) { 393 arcs.put(arc, isError); 394 } 395 396 public void addArc(String arc) { 397 addArc(arc, false); 398 } 399 400 public void setName(String name) { 401 this.name = name; 402 } 403 404 public void setType(String type) { 405 this.type = type; 406 } 407 408 public void setLocation(Point loc) { 409 this.loc = loc; 410 } 411 412 public void setLocation(double x, double y) { 413 loc.setLocation(x, y); 414 } 415 416 public void setStatus(WorkflowAction.Status status) { 417 this.status = status; 418 } 419 420 public String getName() { 421 return name; 422 } 423 424 public String getType() { 425 return type; 426 } 427 428 public HashMap<String, Boolean> getArcs() { 429 return arcs; 430 } 431 432 public Point getLocation() { 433 return loc; 434 } 435 436 public WorkflowAction.Status getStatus() { 437 return status; 438 } 439 440 @Override 441 public String toString() { 442 StringBuilder s = new StringBuilder(); 443 444 s.append("Node: ").append(name).append("\t"); 445 s.append("Type: ").append(type).append("\t"); 446 s.append("Location: (").append(loc.getX()).append(", ").append(loc.getY()).append(")\t"); 447 s.append("Status: ").append(status).append("\n"); 448 Iterator<Map.Entry<String, Boolean>> it = arcs.entrySet().iterator(); 449 while(it.hasNext()) { 450 Map.Entry<String, Boolean> entry = it.next(); 451 452 s.append("\t").append(entry.getKey()); 453 if(entry.getValue().booleanValue()) { 454 s.append(" on error\n"); 455 } else { 456 s.append("\n"); 457 } 458 } 459 460 return s.toString(); 461 } 462 } 463 464 private class NodeFontTransformer implements Transformer<OozieWFNode, Font> { 465 private final Font font = new Font("Default", Font.BOLD, 15); 466 467 @Override 468 public Font transform(OozieWFNode node) { 469 return font; 470 } 471 } 472 473 private class ArrowShapeTransformer implements Transformer<Context<Graph<OozieWFNode, String>, String>, Shape> { 474 private final Shape arrow = ArrowFactory.getWedgeArrow(10.0f, 20.0f); 475 476 @Override 477 public Shape transform(Context<Graph<OozieWFNode, String>, String> i) { 478 return arrow; 479 } 480 } 481 482 private class ArcPaintTransformer implements Transformer<String, Paint> { 483 // Paint based on transition 484 @Override 485 public Paint transform(String arc) { 486 int sep = arc.indexOf("-->"); 487 String source = arc.substring(0, sep); 488 String target = arc.substring(sep + 3); 489 OozieWFNode src = tags.get(source); 490 OozieWFNode tgt = tags.get(target); 491 492 if(src.getType().equals("start")) { 493 if(tgt.getStatus() == null) { 494 return Color.LIGHT_GRAY; 495 } else { 496 return Color.GREEN; 497 } 498 } 499 500 if(src.getArcs().get(target)) { 501 // Dealing with error transition (i.e. target is error) 502 if(src.getStatus() == null) { 503 return Color.LIGHT_GRAY; 504 } 505 switch(src.getStatus()) { 506 case KILLED: 507 case ERROR: 508 case FAILED: 509 return Color.RED; 510 default: 511 return Color.LIGHT_GRAY; 512 } 513 } else { 514 // Non-error 515 if(src.getType().equals("decision")) { 516 // Check for target too 517 if(tgt.getStatus() != null) { 518 return Color.GREEN; 519 } else { 520 return Color.LIGHT_GRAY; 521 } 522 } else { 523 if(src.getStatus() == null) { 524 return Color.LIGHT_GRAY; 525 } 526 switch(src.getStatus()) { 527 case OK: 528 case DONE: 529 case END_RETRY: 530 case END_MANUAL: 531 return Color.GREEN; 532 default: 533 return Color.LIGHT_GRAY; 534 } 535 } 536 } 537 } 538 } 539 540 private class NodeStrokeTransformer implements Transformer<OozieWFNode, Stroke> { 541 private final Stroke stroke1 = new BasicStroke(2.0f); 542 private final Stroke stroke2 = new BasicStroke(4.0f); 543 544 @Override 545 public Stroke transform(OozieWFNode node) { 546 if(node.getType().equals("start") 547 || node.getType().equals("end") 548 || node.getType().equals("kill")) { 549 return stroke2; 550 } 551 return stroke1; 552 } 553 } 554 555 private class NodeLabelTransformer implements Transformer<OozieWFNode, String> { 556 /* 557 * 20 chars in rectangle in 2 rows max 558 * 14 chars in diamond in 2 rows max 559 * 9 in triangle in 2 rows max 560 * 8 in invtriangle in 2 rows max 561 * 8 in circle in 2 rows max 562 */ 563 @Override 564 public String transform(OozieWFNode node) { 565 //return node.getType(); 566 String name = node.getName(); 567 String type = node.getType(); 568 StringBuilder s = new StringBuilder(); 569 if(type.equals("decision")) { 570 if(name.length() <= 14) { 571 return name; 572 } else { 573 s.append("<html>").append(name.substring(0, 12)).append("-<br />"); 574 if(name.substring(13).length() > 14) { 575 s.append(name.substring(12, 25)).append("..."); 576 } else { 577 s.append(name.substring(12)); 578 } 579 s.append("</html>"); 580 return s.toString(); 581 } 582 } else if(type.equals("fork")) { 583 if(name.length() <= 9) { 584 return "<html><br />" + name + "</html>"; 585 } else { 586 s.append("<html><br />").append(name.substring(0, 7)).append("-<br />"); 587 if(name.substring(8).length() > 9) { 588 s.append(name.substring(7, 15)).append("..."); 589 } else { 590 s.append(name.substring(7)); 591 } 592 s.append("</html>"); 593 return s.toString(); 594 } 595 } else if(type.equals("join")) { 596 if(name.length() <= 8) { 597 return "<html>" + name + "</html>"; 598 } else { 599 s.append("<html>").append(name.substring(0, 6)).append("-<br />"); 600 if(name.substring(7).length() > 8) { 601 s.append(name.substring(6, 13)).append("..."); 602 } else { 603 s.append(name.substring(6)); 604 } 605 s.append("</html>"); 606 return s.toString(); 607 } 608 } else if(type.equals("start") 609 || type.equals("end") 610 || type.equals("kill")) { 611 if(name.length() <= 8) { 612 return "<html>" + name + "</html>"; 613 } else { 614 s.append("<html>").append(name.substring(0, 6)).append("-<br />"); 615 if(name.substring(7).length() > 8) { 616 s.append(name.substring(6, 13)).append("..."); 617 } else { 618 s.append(name.substring(6)); 619 } 620 s.append("</html>"); 621 return s.toString(); 622 } 623 }else { 624 if(name.length() <= 20) { 625 return name; 626 } else { 627 s.append("<html>").append(name.substring(0, 18)).append("-<br />"); 628 if(name.substring(19).length() > 20) { 629 s.append(name.substring(18, 37)).append("..."); 630 } else { 631 s.append(name.substring(18)); 632 } 633 s.append("</html>"); 634 return s.toString(); 635 } 636 } 637 } 638 } 639 640 private class NodePaintTransformer implements Transformer<OozieWFNode, Paint> { 641 @Override 642 public Paint transform(OozieWFNode node) { 643 WorkflowJob.Status jobStatus = job.getStatus(); 644 if(node.getType().equals("start")) { 645 return Color.WHITE; 646 } else if(node.getType().equals("end")) { 647 if(jobStatus == WorkflowJob.Status.SUCCEEDED) { 648 return Color.GREEN; 649 } 650 return Color.BLACK; 651 } else if(node.getType().equals("kill")) { 652 if(jobStatus == WorkflowJob.Status.FAILED 653 || jobStatus == WorkflowJob.Status.KILLED) { 654 return Color.RED; 655 } 656 return Color.WHITE; 657 } 658 659 // Paint based on status for rest 660 WorkflowAction.Status status = node.getStatus(); 661 if(status == null) { 662 return Color.LIGHT_GRAY; 663 } 664 switch(status) { 665 case OK: 666 case DONE: 667 case END_RETRY: 668 case END_MANUAL: 669 return Color.GREEN; 670 case PREP: 671 case RUNNING: 672 case USER_RETRY: 673 case START_RETRY: 674 case START_MANUAL: 675 return Color.YELLOW; 676 case KILLED: 677 case ERROR: 678 case FAILED: 679 return Color.RED; 680 default: 681 return Color.LIGHT_GRAY; 682 } 683 } 684 } 685 686 private class NodeShapeTransformer implements Transformer<OozieWFNode, Shape> { 687 private final Ellipse2D.Double circle = new Ellipse2D.Double(-40, -40, 80, 80); 688 private final Rectangle rect = new Rectangle(-100, -30, 200, 60); 689 private final Polygon diamond = new Polygon(new int[]{-75, 0, 75, 0}, new int[]{0, 75, 0, -75}, 4); 690 private final Polygon triangle = new Polygon(new int[]{-85, 85, 0}, new int[]{0, 0, -148}, 3); 691 private final Polygon invtriangle = new Polygon(new int[]{-85, 85, 0}, new int[]{0, 0, 148}, 3); 692 693 @Override 694 public Shape transform(OozieWFNode node) { 695 if("start".equals(node.getType()) 696 || "end".equals(node.getType()) 697 || "kill".equals(node.getType())) { 698 return circle; 699 } 700 if("fork".equals(node.getType())) { 701 return triangle; 702 } 703 if("join".equals(node.getType())) { 704 return invtriangle; 705 } 706 if("decision".equals(node.getType())) { 707 return diamond; 708 } 709 return rect; // All action nodes 710 } 711 } 712 713 private class ArcStrokeTransformer implements Transformer<String, Stroke> { 714 private final Stroke stroke1 = new BasicStroke(2.0f); 715 private final Stroke dashed = new BasicStroke(1.0f, BasicStroke.CAP_BUTT, BasicStroke.JOIN_MITER, 10.0f, new float[] {10.0f}, 0.0f); 716 717 // Draw based on transition 718 @Override 719 public Stroke transform(String arc) { 720 int sep = arc.indexOf("-->"); 721 String source = arc.substring(0, sep); 722 String target = arc.substring(sep + 3); 723 OozieWFNode src = tags.get(source); 724 if(src.getArcs().get(target)) { 725 if(src.getStatus() == null) { 726 return dashed; 727 } 728 switch(src.getStatus()) { 729 case KILLED: 730 case ERROR: 731 case FAILED: 732 return stroke1; 733 default: 734 return dashed; 735 } 736 } else { 737 return stroke1; 738 } 739 } 740 } 741 } 742}