You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by ar...@apache.org on 2023/05/17 21:53:21 UTC

[systemds] branch main updated: [SYSTEMDS-3564] Cost-based operator linearization

This is an automated email from the ASF dual-hosted git repository.

arnabp20 pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/systemds.git


The following commit(s) were added to refs/heads/main by this push:
     new f399f39a63 [SYSTEMDS-3564] Cost-based operator linearization
f399f39a63 is described below

commit f399f39a635fa582860f2082946b57ef08b29eb3
Author: Arnab Phani <ph...@gmail.com>
AuthorDate: Wed May 17 23:53:05 2023 +0200

    [SYSTEMDS-3564] Cost-based operator linearization
    
    This patch adds an initial version of cost-based instruction ordering.
    The idea is to apply a non-recursive exhaustive search to find all the
    possible orders and select the order with the minimum cost under memory
    constraints. We propagate the memory and compute estimates from HOPs
    to LOPs. Moreover, we simplify the DAG by removing literals and transient
    read, writes to reduce the search space.
    
    Closes #1827
---
 .../apache/sysds/conf/ConfigurationManager.java    |   8 +-
 src/main/java/org/apache/sysds/hops/Hop.java       |  12 ++
 .../java/org/apache/sysds/hops/OptimizerUtils.java |   6 +
 src/main/java/org/apache/sysds/lops/Lop.java       |  61 +++++-
 .../compile/linearization/CostBasedLinearize.java  | 228 +++++++++++++++++++++
 .../lops/compile/linearization/ILinearize.java     |   6 +-
 .../test/functions/async/CostBasedOrderTest.java   |  78 +++++++
 .../scripts/functions/async/CostBasedOrder1.dml    |  29 +++
 8 files changed, 424 insertions(+), 4 deletions(-)

diff --git a/src/main/java/org/apache/sysds/conf/ConfigurationManager.java b/src/main/java/org/apache/sysds/conf/ConfigurationManager.java
index a2855e0eed..3086ada163 100644
--- a/src/main/java/org/apache/sysds/conf/ConfigurationManager.java
+++ b/src/main/java/org/apache/sysds/conf/ConfigurationManager.java
@@ -262,6 +262,10 @@ public class ConfigurationManager{
 			|| OptimizerUtils.MAX_PARALLELIZE_ORDER);
 	}
 
+	public static boolean isAutoLinearizationEnabled() {
+		return OptimizerUtils.COST_BASED_ORDERING;
+	}
+
 	public static boolean isParallelIOEnabled(){
 		return getDMLConfig().getBooleanValue(DMLConfig.CP_PARALLEL_IO);
 	}
@@ -276,7 +280,9 @@ public class ConfigurationManager{
 	}
 
 	public static ILinearize.DagLinearization getLinearizationOrder() {
-		if (OptimizerUtils.MAX_PARALLELIZE_ORDER)
+		if (OptimizerUtils.COST_BASED_ORDERING)
+			return ILinearize.DagLinearization.AUTO;
+		else if (OptimizerUtils.MAX_PARALLELIZE_ORDER)
 			return ILinearize.DagLinearization.MAX_PARALLELIZE;
 		else
 			return ILinearize.DagLinearization
diff --git a/src/main/java/org/apache/sysds/hops/Hop.java b/src/main/java/org/apache/sysds/hops/Hop.java
index 890948562d..bb46b7c2e3 100644
--- a/src/main/java/org/apache/sysds/hops/Hop.java
+++ b/src/main/java/org/apache/sysds/hops/Hop.java
@@ -36,6 +36,7 @@ import org.apache.sysds.common.Types.OpOp2;
 import org.apache.sysds.common.Types.OpOpData;
 import org.apache.sysds.common.Types.ValueType;
 import org.apache.sysds.conf.ConfigurationManager;
+import org.apache.sysds.hops.cost.ComputeCost;
 import org.apache.sysds.hops.cost.FederatedCost;
 import org.apache.sysds.hops.recompile.Recompiler;
 import org.apache.sysds.hops.recompile.Recompiler.ResetType;
@@ -410,6 +411,11 @@ public abstract class Hop implements ParseInfo {
 			getLops().setFederatedOutput(_federatedOutput);
 		if ( prefetchActivated() )
 			getLops().activatePrefetch();
+
+		//propagate compute and memory estimates to lops
+		//FIXME: Compute cost. Handle multiple Lops from one Hop case
+		if (ConfigurationManager.isAutoLinearizationEnabled())
+			setMemoryAndComputeEstimates(getLops());
 		
 		//Step 1: construct reblock lop if required (output of hop)
 		constructAndSetReblockLopIfRequired();
@@ -1697,6 +1703,12 @@ public abstract class Hop implements ParseInfo {
 		lop.setPrivacyConstraint(getPrivacy());
 	}
 
+	protected void setMemoryAndComputeEstimates(Lop lop) {
+		lop.setMemoryEstimates(getOutputMemEstimate(), getMemEstimate(),
+			getIntermediateMemEstimate(), getSpBroadcastSize());
+		lop.setComputeEstimate(ComputeCost.getHOPComputeCost(this));
+	}
+
 	/**
 	 * Set parse information.
 	 *
diff --git a/src/main/java/org/apache/sysds/hops/OptimizerUtils.java b/src/main/java/org/apache/sysds/hops/OptimizerUtils.java
index 4ab7c33dbd..19cd080525 100644
--- a/src/main/java/org/apache/sysds/hops/OptimizerUtils.java
+++ b/src/main/java/org/apache/sysds/hops/OptimizerUtils.java
@@ -299,6 +299,12 @@ public class OptimizerUtils
 	 */
 	public static boolean MAX_PARALLELIZE_ORDER = false;
 
+	/**
+	 * Cost-based instruction ordering to minimize total execution time under
+	 * the constraint of available memory.
+	 */
+	public static boolean COST_BASED_ORDERING = false;
+
 	//////////////////////
 	// Optimizer levels //
 	//////////////////////
diff --git a/src/main/java/org/apache/sysds/lops/Lop.java b/src/main/java/org/apache/sysds/lops/Lop.java
index b768ded9ad..3d84348d80 100644
--- a/src/main/java/org/apache/sysds/lops/Lop.java
+++ b/src/main/java/org/apache/sysds/lops/Lop.java
@@ -27,6 +27,7 @@ import org.apache.sysds.common.Types.DataType;
 import org.apache.sysds.common.Types.ValueType;
 import org.apache.sysds.hops.AggBinaryOp.SparkAggType;
 import org.apache.sysds.common.Types.ExecType;
+import org.apache.sysds.hops.OptimizerUtils;
 import org.apache.sysds.lops.compile.Dag;
 import org.apache.sysds.runtime.instructions.fed.FEDInstruction.FederatedOutput;
 import org.apache.sysds.runtime.privacy.PrivacyConstraint;
@@ -155,7 +156,34 @@ public abstract class Lop
 	 * Examples include spark unary aggregate, mapmm, prefetch
 	 */
 	protected boolean _asynchronous = false;
-	
+
+	/**
+	 * Estimated size for the output produced by this Lop in bytes.
+	 */
+	protected double _outputMemEstimate = OptimizerUtils.INVALID_SIZE;
+
+	/*
+	 * Estimated size for the entire operation represented by this Lop
+	 * It includes the memory required for all inputs as well as the output
+	 * For Spark collects, _memEstimate equals _outputMemEstimate.
+	 */
+	protected double _memEstimate = OptimizerUtils.INVALID_SIZE;
+
+	/**
+	 * Estimated size for the intermediates produced by this Lop in bytes.
+	 */
+	protected double _processingMemEstimate = 0;
+
+	/**
+	 * Estimated size for the broadcast partitions.
+	 */
+	protected double _spBroadcastMemEstimate = 0;
+
+	/*
+	 * Compute cost for this Lop based on the number of floating point operations per
+	 * output cell and the total number of output cells.
+	 */
+	protected double _computeCost = 0;
 
 	/**
 	 * Constructor to be invoked by base class.
@@ -310,6 +338,14 @@ public abstract class Lop
 		}
 	}
 
+	public void replaceAllInputs(ArrayList<Lop> newInputs) {
+		inputs = newInputs;
+	}
+
+	public void replaceAllOutputs(ArrayList<Lop> newOutputs) {
+		outputs = newOutputs;
+	}
+
 	public void removeInput(Lop op) {
 		inputs.remove(op);
 	}
@@ -378,6 +414,29 @@ public abstract class Lop
 		return _asynchronous;
 	}
 
+	public void setMemoryEstimates(double outMem, double totMem, double interMem, double bcMem) {
+		_outputMemEstimate = outMem;
+		_memEstimate = totMem;
+		_processingMemEstimate = interMem;
+		_spBroadcastMemEstimate = bcMem;
+	}
+
+	public double getTotalMemoryEstimate() {
+		return _memEstimate;
+	}
+
+	public double getOutputMemoryEstimate() {
+		return _outputMemEstimate;
+	}
+
+	public void setComputeEstimate(double compCost) {
+		_computeCost = compCost;
+	}
+
+	public double getComputeEstimate() {
+		return _computeCost;
+	}
+
 	/**
 	 * Method to have Lops print their state. This is for debugging purposes.
 	 */
diff --git a/src/main/java/org/apache/sysds/lops/compile/linearization/CostBasedLinearize.java b/src/main/java/org/apache/sysds/lops/compile/linearization/CostBasedLinearize.java
new file mode 100644
index 0000000000..690e11866f
--- /dev/null
+++ b/src/main/java/org/apache/sysds/lops/compile/linearization/CostBasedLinearize.java
@@ -0,0 +1,228 @@
+/*
+ * 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.sysds.lops.compile.linearization;
+
+import org.apache.sysds.common.Types;
+import org.apache.sysds.hops.OptimizerUtils;
+import org.apache.sysds.lops.Data;
+import org.apache.sysds.lops.Lop;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Stack;
+import java.util.stream.Collectors;
+
+public class CostBasedLinearize
+{
+	public static List<Lop> getBestOrder(List<Lop> lops)
+	{
+		// Simplify DAG by removing literals and transient inputs and outputs
+		List<Lop> removedLeaves = new ArrayList<>();
+		List<Lop> removedRoots = new ArrayList<>();
+		HashMap<Long, ArrayList<Lop>> removedInputs = new HashMap<>();
+		HashMap<Long, ArrayList<Lop>> removedOutputs = new HashMap<>();
+		simplifyDag(lops, removedLeaves, removedRoots, removedInputs, removedOutputs);
+		// TODO: Partition the DAG if connected by a single node. Optimize separately
+
+		// Collect the leaf nodes of the simplified DAG
+		List<Lop> leafNodes = lops.stream().filter(l -> l.getInputs().isEmpty()).collect(Collectors.toList());
+
+		// For each leaf, find all possible orders starting from the given leaf
+		List<Order> finalOrders = new ArrayList<>();
+		for (Lop leaf : leafNodes)
+			generateOrders(leaf, leafNodes, finalOrders, lops.size());
+
+		// TODO: Handle distributed and GPU operators (0 compute cost, memory overhead on collect)
+		// TODO: Asynchronous operators (max of compute costs, total operation memory overhead)
+
+		// TODO: Select the order with minimum compute cost and buffer pool evictions
+		int randInd = (int) (Math.random() * finalOrders.size());
+		List<Lop> best = finalOrders.get(randInd).getOrder();
+
+		// Add the removed leaf and root nodes back to the list
+		addRemovedNodes(best, removedLeaves, removedRoots, removedInputs, removedOutputs);
+
+		return best;
+	}
+
+	private static void generateOrders(Lop leaf, List<Lop> leafNodes, List<Order> finalOrders, int count)
+	{
+		// Create a stack to store the partial solutions
+		Stack<Order> stack = new Stack<>();
+		stack.push(new Order(leaf));
+
+		while (!stack.isEmpty())
+		{
+			// Pop a partial order of Lops
+			Order partialOrder = stack.pop();
+
+			// If the partial order contains all nodes, move it to the full solution list
+			if (partialOrder.size() == count) {
+				finalOrders.add(partialOrder);
+				continue;
+			}
+
+			// Create new partial orders and push to the stack
+			List<Lop> distinctOutputs = new ArrayList<>();
+			// Collect the distinct set of outputs of the already listed nodes
+			for (Lop lop : partialOrder.getOrder()) {
+				for(Lop out : lop.getOutputs()) {
+					if(!out.isVisited() && allInputsLinearized(out, partialOrder) && !partialOrder.contains(out)) {
+						out.setVisited();
+						distinctOutputs.add(out);
+					}
+				}
+			}
+			// Create new partial orders with the outputs of the already listed nodes
+			for (Lop out : distinctOutputs) {
+				out.resetVisitStatus();
+				stack.push(copyAndAdd(partialOrder, out, true));
+			}
+
+			// Create new partial orders with the disconnected leaves
+			for (Lop otherLeaf : leafNodes) {
+				if (!partialOrder.contains(otherLeaf)) {
+					stack.push(copyAndAdd(partialOrder, otherLeaf, false));
+				}
+			}
+		}
+	}
+
+	private static boolean allInputsLinearized(Lop lop, Order partialOrder) {
+		List<Lop> order = partialOrder.getOrder();
+		for (Lop input : lop.getInputs()) {
+			if (!order.contains(input))
+				return false;
+		}
+		return true;
+	}
+
+	private static Order copyAndAdd(Order partialOrder, Lop node, boolean allInputsLinearized) {
+		Order newEntry = new Order(partialOrder);
+		// Add the new operator and maintain memory and compute estimates
+		newEntry.addOperator(node, allInputsLinearized);
+		return newEntry;
+	}
+
+	private static void simplifyDag(List<Lop> lops, List<Lop> removedLeaves, List<Lop> removedRoots,
+		HashMap<Long, ArrayList<Lop>> removedInputs, HashMap<Long, ArrayList<Lop>> removedOutputs) {
+		// Store the removed nodes and the full input/output arrays (order preserving)
+		for (Lop lop : lops) {
+			if (lop.getInputs().isEmpty()
+				&& ((lop instanceof Data && ((Data) lop).isTransientRead())
+				|| lop.getDataType() == Types.DataType.SCALAR)) {
+				removedLeaves.add(lop);
+				for (Lop out : lop.getOutputs()) {
+					removedInputs.putIfAbsent(out.getID(), new ArrayList<>(out.getInputs()));
+					out.removeInput(lop);
+				}
+			}
+			if (lop.getOutputs().isEmpty()
+				&& lop instanceof Data && ((Data) lop).isTransientWrite()) {
+				removedRoots.add(lop);
+				for (Lop in : lop.getInputs()) {
+					removedOutputs.putIfAbsent(in.getID(), new ArrayList<>(in.getOutputs()));
+					in.removeOutput(lop);
+				}
+			}
+		}
+		// Remove the insignificant nodes from the main list
+		lops.removeAll(removedLeaves);
+		lops.removeAll(removedRoots);
+	}
+
+	private static void addRemovedNodes(List<Lop> lops, List<Lop> removedLeaves, List<Lop> removedRoots,
+		// Add the nodes, removed during simplification back
+		HashMap<Long, ArrayList<Lop>> removedInputs, HashMap<Long, ArrayList<Lop>> removedOutputs) {
+		for (Lop leaf : removedLeaves)
+			leaf.getOutputs().forEach(out -> out.replaceAllInputs(removedInputs.get(out.getID())));
+		lops.addAll(0, removedLeaves);
+
+		for (Lop root : removedRoots)
+			root.getInputs().forEach(in -> in.replaceAllOutputs(removedOutputs.get(in.getID())));
+		lops.addAll(removedRoots);
+	}
+
+	private static class Order
+	{
+		private List<Lop> _order;
+		private double _pinnedMemEstimate;
+		private double _bufferpoolEstimate;
+		private int _numEvictions;
+		private double _computeCost;
+
+		public Order(List<Lop> lops, double pin, double bp, double comp) {
+			_order = new ArrayList<>(lops);
+			_pinnedMemEstimate = pin;
+			_bufferpoolEstimate = bp;
+			_numEvictions = 0;
+			_computeCost = comp;
+		}
+
+		public Order(Lop lop) {
+			// Initiate the memory estimates for the first operator
+			this(Arrays.asList(lop), lop.getOutputMemoryEstimate(), 0, lop.getComputeEstimate());
+		}
+
+		public Order(Order that) {
+			_order = that.getOrder();
+			_pinnedMemEstimate = that._pinnedMemEstimate;
+			_bufferpoolEstimate = that._bufferpoolEstimate;
+			_numEvictions = that._numEvictions;
+			_computeCost = that._computeCost;
+		}
+
+		public void addOperator(Lop lop, boolean allInputsLinearized) {
+			_order.add(lop);
+			// Update total compute cost for this partial order
+			_computeCost += lop.getComputeEstimate();
+			// Estimate buffer pool state after executing this operator
+			_bufferpoolEstimate += lop.getOutputMemoryEstimate();
+			if (allInputsLinearized) {
+				lop.getInputs().forEach(in ->_bufferpoolEstimate -= in.getOutputMemoryEstimate());
+				_bufferpoolEstimate = _bufferpoolEstimate < 0 ? 0 : _bufferpoolEstimate;
+			}
+			// Maintain total eviction count for this order
+			if (_bufferpoolEstimate > OptimizerUtils.getBufferPoolLimit())
+				_numEvictions++;
+			// TODO: Add IO time to compute cost for evictions
+			// Estimate operational memory state during the execution of this operator
+			_pinnedMemEstimate = lop.getTotalMemoryEstimate();
+		}
+
+		protected List<Lop> getOrder() {
+			return _order;
+		}
+
+		protected double getComputeCost() {
+			return _computeCost;
+		}
+
+		protected boolean contains(Lop lop) {
+			return _order.contains(lop);
+		}
+
+		protected int size() {
+			return _order.size();
+		}
+	}
+}
diff --git a/src/main/java/org/apache/sysds/lops/compile/linearization/ILinearize.java b/src/main/java/org/apache/sysds/lops/compile/linearization/ILinearize.java
index e5ee982c4f..3c0aa61692 100644
--- a/src/main/java/org/apache/sysds/lops/compile/linearization/ILinearize.java
+++ b/src/main/java/org/apache/sysds/lops/compile/linearization/ILinearize.java
@@ -62,11 +62,11 @@ import org.apache.sysds.lops.UnaryCP;
  *
  * https://en.wikipedia.org/wiki/Linearizability#Linearization_points
  */
-public interface ILinearize {
+public class ILinearize {
 	public static Log LOG = LogFactory.getLog(ILinearize.class.getName());
 
 	public enum DagLinearization {
-		DEPTH_FIRST, BREADTH_FIRST, MIN_INTERMEDIATE, MAX_PARALLELIZE
+		DEPTH_FIRST, BREADTH_FIRST, MIN_INTERMEDIATE, MAX_PARALLELIZE, AUTO
 	}
 
 	public static List<Lop> linearize(List<Lop> v) {
@@ -76,6 +76,8 @@ public interface ILinearize {
 			switch(linearization) {
 				case MAX_PARALLELIZE:
 					return doMaxParallelizeSort(v);
+				case AUTO:
+					return CostBasedLinearize.getBestOrder(v);
 				case MIN_INTERMEDIATE:
 					return doMinIntermediateSort(v);
 				case BREADTH_FIRST:
diff --git a/src/test/java/org/apache/sysds/test/functions/async/CostBasedOrderTest.java b/src/test/java/org/apache/sysds/test/functions/async/CostBasedOrderTest.java
new file mode 100644
index 0000000000..5caf3ac755
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/async/CostBasedOrderTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.sysds.test.functions.async;
+
+import org.apache.sysds.hops.OptimizerUtils;
+import org.apache.sysds.runtime.matrix.data.MatrixValue;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+import org.apache.sysds.test.TestUtils;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+public class CostBasedOrderTest extends AutomatedTestBase {
+
+	protected static final String TEST_DIR = "functions/async/";
+	protected static final String TEST_NAME = "CostBasedOrder";
+	protected static final int TEST_VARIANTS = 1;
+	protected static String TEST_CLASS_DIR = TEST_DIR + CostBasedOrderTest.class.getSimpleName() + "/";
+
+	@Override
+	public void setUp() {
+		TestUtils.clearAssertionInformation();
+		for(int i=1; i<=TEST_VARIANTS; i++)
+			addTestConfiguration(TEST_NAME+i, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME+i));
+	}
+
+	@Test
+	public void testlmds() {
+		runTest(TEST_NAME+"1");
+	}
+
+	public void runTest(String testname) {
+		getAndLoadTestConfiguration(testname);
+		fullDMLScriptName = getScript();
+
+		List<String> proArgs = new ArrayList<>();
+
+		proArgs.add("-explain");
+		proArgs.add("-stats");
+		proArgs.add("-args");
+		proArgs.add(output("R"));
+		programArgs = proArgs.toArray(new String[proArgs.size()]);
+
+		runTest(true, EXCEPTION_NOT_EXPECTED, null, -1);
+		HashMap<MatrixValue.CellIndex, Double> R = readDMLScalarFromOutputDir("R");
+
+		OptimizerUtils.COST_BASED_ORDERING = true;
+		runTest(true, EXCEPTION_NOT_EXPECTED, null, -1);
+		HashMap<MatrixValue.CellIndex, Double> R_mp = readDMLScalarFromOutputDir("R");
+		OptimizerUtils.COST_BASED_ORDERING = false;
+
+		//compare matrices
+		boolean matchVal = TestUtils.compareMatrices(R, R_mp, 1e-6, "Origin", "withMaxParallelize");
+		if (!matchVal)
+			System.out.println("Value w/ depth first"+R+" w/ cost-based"+R_mp);
+	}
+
+}
diff --git a/src/test/scripts/functions/async/CostBasedOrder1.dml b/src/test/scripts/functions/async/CostBasedOrder1.dml
new file mode 100644
index 0000000000..c12c3d51ee
--- /dev/null
+++ b/src/test/scripts/functions/async/CostBasedOrder1.dml
@@ -0,0 +1,29 @@
+#-------------------------------------------------------------
+#
+# 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.
+#
+#-------------------------------------------------------------
+
+X = rand(rows=1000, cols=10, seed=42);
+y = rand(rows=1000, cols=1, seed=42);
+A = (t(X) %*% X) + diag(matrix(0.001, rows=ncol(X), cols=1));
+b = t(X) %*% y;
+beta = solve(A, b);
+
+R = sum(beta);
+write(R, $1, format="text");