You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by mb...@apache.org on 2020/09/13 12:13:58 UTC

[systemds] branch master updated: [SYSTEMDS-2605] Fine-grained privacy constraints and propagation

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9139fe0  [SYSTEMDS-2605] Fine-grained privacy constraints and propagation
9139fe0 is described below

commit 9139fe023e1c6647d48c5fdf4884cb62d6ba5b2b
Author: sebwrede <sw...@know-center.at>
AuthorDate: Sun Sep 13 14:10:30 2020 +0200

    [SYSTEMDS-2605] Fine-grained privacy constraints and propagation
    
    - Add Initial Implementation of FineGrainedPrivacyConstraint
    - Add Unfinished Basic Implementation of R Tree
    - Add FineGrainedPrivacy Implementation Based on List
    - Raise Operator Field to Instruction Class and Prepare
    PrivacyPropagator for Fine-Grained Constraints
    - Add Initial Fine-Grained Privacy Propagation
    - Add JavaDoc Details
    - Add CovarianceCPInstruction to Privacy Propagator
    - Propagator will throw exception if privacy constraint activated.
    - Add Write And Read of FineGrainedPrivacy
    - Edit PrivacyConstraint to Implement Externalizable
      So that it does not use Serializable interface to serialize DataRange
    instances
    - Adapt Privacy Propagation and Constraint Handling To the New Federated
    Architecture
    - Add Documentation
    - Edit Long-Conversion For SetPrivacy in DataExpression
    
    Closes #1051.
---
 .../org/apache/sysds/parser/DataExpression.java    |  39 ++-
 .../runtime/compress/CompressedMatrixBlock.java    |   6 +-
 .../federated/FederatedWorkerHandler.java          |   6 +-
 .../sysds/runtime/instructions/Instruction.java    |  10 +
 .../runtime/instructions/cp/CPInstruction.java     |   3 +-
 .../sysds/runtime/instructions/cp/CPOperand.java   |  13 +-
 .../instructions/cp/ComputationCPInstruction.java  |   4 +
 .../cp/MultiReturnBuiltinCPInstruction.java        |   5 +
 ...ltiReturnParameterizedBuiltinCPInstruction.java |   5 +
 .../runtime/instructions/cp/SqlCPInstruction.java  |   4 +
 .../runtime/instructions/fed/FEDInstruction.java   |   3 +-
 .../runtime/instructions/gpu/GPUInstruction.java   |   9 +-
 .../runtime/instructions/spark/SPInstruction.java  |   7 +-
 .../sysds/runtime/privacy/PrivacyConstraint.java   | 214 ++++++++++++-
 .../sysds/runtime/privacy/PrivacyPropagator.java   | 335 ++++++++++++++++-----
 .../apache/sysds/runtime/privacy/PrivacyUtils.java |  79 +++++
 .../runtime/privacy/finegrained/DataRange.java     | 114 +++++++
 .../privacy/finegrained/FineGrainedPrivacy.java    |  80 +++++
 .../finegrained/FineGrainedPrivacyList.java        | 126 ++++++++
 .../privacy/finegrained/FineGrainedPrivacyMap.java | 105 +++++++
 .../org/apache/sysds/runtime/util/HDFSTool.java    |  10 +-
 .../org/apache/sysds/test/AutomatedTestBase.java   |   1 -
 .../test/functions/privacy/BuiltinGLMTest.java     |   6 +-
 .../test/functions/privacy/FederatedL2SVMTest.java |  97 +++---
 .../privacy/FederatedWorkerHandlerTest.java        |  54 ++--
 .../functions/privacy/FineGrainedPrivacyTest.java  | 191 ++++++++++++
 .../sysds/test/functions/privacy/GLMTest.java      |   6 +-
 .../MatrixMultiplicationPropagationTest.java       |  24 +-
 .../functions/privacy/PrivacyPropagatorTest.java   | 107 +++++++
 .../test/functions/privacy/ReadWriteTest.java      | 178 +++++++++++
 .../MatrixMultiplicationPropagationTest.dml        |   2 -
 ...ationPropagationTest.dml => ReadWriteTest2.dml} |   7 +-
 32 files changed, 1664 insertions(+), 186 deletions(-)

diff --git a/src/main/java/org/apache/sysds/parser/DataExpression.java b/src/main/java/org/apache/sysds/parser/DataExpression.java
index 8788e0f..f9fe5a4 100644
--- a/src/main/java/org/apache/sysds/parser/DataExpression.java
+++ b/src/main/java/org/apache/sysds/parser/DataExpression.java
@@ -36,6 +36,8 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.wink.json4j.JSONArray;
+import org.apache.wink.json4j.JSONObject;
 import org.apache.sysds.api.DMLScript;
 import org.apache.sysds.common.Types.DataType;
 import org.apache.sysds.common.Types.FileFormat;
@@ -49,12 +51,12 @@ import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.controlprogram.parfor.stat.InfrastructureAnalyzer;
 import org.apache.sysds.runtime.io.FileFormatPropertiesMM;
 import org.apache.sysds.runtime.io.IOUtilFunctions;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint;
 import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+import org.apache.sysds.runtime.privacy.PrivacyUtils;
 import org.apache.sysds.runtime.util.HDFSTool;
 import org.apache.sysds.runtime.util.UtilFunctions;
 import org.apache.sysds.utils.JSONHelper;
-import org.apache.wink.json4j.JSONArray;
-import org.apache.wink.json4j.JSONObject;
 
 public class DataExpression extends DataIdentifier
 {
@@ -103,6 +105,7 @@ public class DataExpression extends DataIdentifier
 	public static final String CREATEDPARAM = "created";
 
 	public static final String PRIVACY = "privacy";
+	public static final String FINE_GRAINED_PRIVACY = "fine_grained_privacy";
 
 	// Parameter names relevant to reading/writing delimited/csv files
 	public static final String DELIM_DELIMITER = "sep";
@@ -137,7 +140,7 @@ public class DataExpression extends DataIdentifier
 			// Parameters related to delimited/csv files.
 			DELIM_FILL_VALUE, DELIM_DELIMITER, DELIM_FILL, DELIM_HAS_HEADER_ROW, DELIM_NA_STRINGS,
 			// Parameters related to privacy
-			PRIVACY));
+			PRIVACY, FINE_GRAINED_PRIVACY));
 
 	/** Valid parameter names in arguments to read instruction */
 	public static final Set<String> READ_VALID_PARAM_NAMES = new HashSet<>(
@@ -2104,12 +2107,13 @@ public class DataExpression extends DataIdentifier
 							addVarParam(key.toString(), doubleId);
 						}
 						else if (key.toString().equalsIgnoreCase(DELIM_NA_STRINGS) 
-								|| key.toString().equalsIgnoreCase(PRIVACY)) {
+								|| key.toString().equalsIgnoreCase(PRIVACY)
+								|| key.toString().equalsIgnoreCase(FINE_GRAINED_PRIVACY)) {
 							String naStrings = null;
 							if ( val instanceof String) {
 								naStrings = val.toString();
 							}
-							else {
+							else if (val instanceof JSONArray) {
 								StringBuilder sb = new StringBuilder();
 								JSONArray valarr = (JSONArray)val;
 								for(int naid=0; naid < valarr.size(); naid++ ) {
@@ -2119,6 +2123,14 @@ public class DataExpression extends DataIdentifier
 								}
 								naStrings = sb.toString();
 							}
+							else if ( val instanceof JSONObject ){
+								JSONObject valJsonObject = (JSONObject)val;
+								naStrings = valJsonObject.toString();
+							}
+							else {
+								throw new ParseException("Type of value " + val 
+									+ " from metadata not recognized by parser.");
+							}
 							StringIdentifier sid = new StringIdentifier(naStrings, this);
 							removeVarParam(key.toString());
 							addVarParam(key.toString(), sid);
@@ -2130,7 +2142,7 @@ public class DataExpression extends DataIdentifier
 					}
 				}
 			}
-    	}
+		}
 	}
 	
 	public JSONObject readMetadataFile(String filename, boolean conditional) 
@@ -2239,10 +2251,17 @@ public class DataExpression extends DataIdentifier
 	 * Sets privacy of identifier if privacy variable parameter is set.  
 	 */
 	private void setPrivacy(){
-		Expression eprivacy = getVarParam("privacy");
-		if ( eprivacy != null ){
-			getOutput().setPrivacy(PrivacyLevel.valueOf(eprivacy.toString()));
+		Expression eprivacy = getVarParam(PRIVACY);
+		Expression eFineGrainedPrivacy = getVarParam(FINE_GRAINED_PRIVACY);
+		if ( eprivacy != null || eFineGrainedPrivacy != null ){
+			PrivacyConstraint privacyConstraint = new PrivacyConstraint();
+			if ( eprivacy != null ){
+				privacyConstraint.setPrivacyLevel(PrivacyLevel.valueOf(eprivacy.toString()));
+			}
+			if ( eFineGrainedPrivacy != null ){
+				PrivacyUtils.setFineGrainedPrivacy(privacyConstraint, eFineGrainedPrivacy);
+			}
+			getOutput().setPrivacy(privacyConstraint);
 		}
 	}
-	
 } // end class
diff --git a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
index a0131b9..dacce77 100644
--- a/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
+++ b/src/main/java/org/apache/sysds/runtime/compress/CompressedMatrixBlock.java
@@ -1143,9 +1143,9 @@ public class CompressedMatrixBlock extends AbstractCompressedMatrixBlock {
 		ret.recomputeNonZeros();
 	}
 
-	private MatrixBlock rightMultByMatrix(List<ColGroup> colGroups, MatrixBlock that, MatrixBlock ret, int k,
-		int numColumns) {
-
+	private static MatrixBlock rightMultByMatrix(List<ColGroup> colGroups,
+		MatrixBlock that, MatrixBlock ret, int k, int numColumns)
+	{
 		// Exchange with rightMultByDenseMatrix.
 
 		ret.reset();
diff --git a/src/main/java/org/apache/sysds/runtime/controlprogram/federated/FederatedWorkerHandler.java b/src/main/java/org/apache/sysds/runtime/controlprogram/federated/FederatedWorkerHandler.java
index 2690ee6..2d6ea04 100644
--- a/src/main/java/org/apache/sysds/runtime/controlprogram/federated/FederatedWorkerHandler.java
+++ b/src/main/java/org/apache/sysds/runtime/controlprogram/federated/FederatedWorkerHandler.java
@@ -41,6 +41,7 @@ import org.apache.sysds.runtime.controlprogram.caching.MatrixObject;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.controlprogram.federated.FederatedRequest.RequestType;
 import org.apache.sysds.runtime.controlprogram.federated.FederatedResponse.ResponseType;
+import org.apache.sysds.runtime.instructions.Instruction;
 import org.apache.sysds.runtime.instructions.InstructionParser;
 import org.apache.sysds.runtime.instructions.cp.Data;
 import org.apache.sysds.runtime.instructions.cp.ListObject;
@@ -280,8 +281,9 @@ public class FederatedWorkerHandler extends ChannelInboundHandlerAdapter {
 		ExecutionContext ec = _ecm.get(request.getTID());
 		BasicProgramBlock pb = new BasicProgramBlock(null);
 		pb.getInstructions().clear();
-		pb.getInstructions().add(InstructionParser
-			.parseSingleInstruction((String)request.getParam(0)));
+		Instruction receivedInstruction = InstructionParser
+			.parseSingleInstruction((String)request.getParam(0));
+		pb.getInstructions().add(receivedInstruction);
 		try {
 			pb.execute(ec); //execute single instruction
 		}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/Instruction.java b/src/main/java/org/apache/sysds/runtime/instructions/Instruction.java
index 2cfb68b..7a2e410 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/Instruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/Instruction.java
@@ -25,6 +25,7 @@ import org.apache.sysds.api.DMLScript;
 import org.apache.sysds.lops.Lop;
 import org.apache.sysds.parser.DataIdentifier;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
+import org.apache.sysds.runtime.matrix.operators.Operator;
 import org.apache.sysds.runtime.privacy.PrivacyConstraint;
 import org.apache.sysds.runtime.privacy.PrivacyPropagator;
 
@@ -39,6 +40,11 @@ public abstract class Instruction
 	}
 	
 	private static final Log LOG = LogFactory.getLog(Instruction.class.getName());
+	protected final Operator _optr;
+
+	protected Instruction(Operator _optr){
+		this._optr = _optr;
+	}
 
 	public static final String OPERAND_DELIM = Lop.OPERAND_DELIMITOR;
 	public static final String DATATYPE_PREFIX = Lop.DATATYPE_PREFIX;
@@ -136,6 +142,10 @@ public abstract class Instruction
 	public PrivacyConstraint getPrivacyConstraint(){
 		return privacyConstraint;
 	}
+
+	public Operator getOperator() {
+		return _optr;
+	}
 	
 	/**
 	 * Getter for instruction line number
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/CPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/CPInstruction.java
index 7e5f359..8ce3dec 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/CPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/CPInstruction.java
@@ -43,7 +43,6 @@ public abstract class CPInstruction extends Instruction
 		StringInit, CentralMoment, Covariance, UaggOuterChain, Dnn, Sql }
 
 	protected final CPType _cptype;
-	protected final Operator _optr;
 	protected final boolean _requiresLabelUpdate;
 
 	protected CPInstruction(CPType type, String opcode, String istr) {
@@ -51,8 +50,8 @@ public abstract class CPInstruction extends Instruction
 	}
 
 	protected CPInstruction(CPType type, Operator op, String opcode, String istr) {
+		super(op);
 		_cptype = type;
-		_optr = op;
 		instString = istr;
 
 		// prepare opcode and update requirement for repeated usage
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/CPOperand.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/CPOperand.java
index 3afb681..30ded87 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/CPOperand.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/CPOperand.java
@@ -26,15 +26,16 @@ import org.apache.sysds.common.Types.ValueType;
 import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.instructions.Instruction;
 import org.apache.sysds.runtime.instructions.InstructionUtils;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint;
 
-
-public class CPOperand 
+public class CPOperand
 {
 	private String _name;
 	private ValueType _valueType;
 	private DataType _dataType;
 	private boolean _isLiteral;
 	private ScalarObject _literal;
+	private PrivacyConstraint _privacyConstraint;
 	
 	public CPOperand() {
 		this("", ValueType.UNKNOWN, DataType.UNKNOWN);
@@ -160,6 +161,14 @@ public class CPOperand
 		}
 	}
 
+	public PrivacyConstraint getPrivacyConstraint() {
+		return _privacyConstraint;
+	}
+
+	public void setPrivacyConstraint(PrivacyConstraint privacyConstraint) {
+		_privacyConstraint = privacyConstraint;
+	}
+
 	@Override
 	public String toString() {
 		return ToStringBuilder.reflectionToString(this, ToStringStyle.SHORT_PREFIX_STYLE);
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/ComputationCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/ComputationCPInstruction.java
index 5bdc1d6..76c1617 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/ComputationCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/ComputationCPInstruction.java
@@ -59,6 +59,10 @@ public abstract class ComputationCPInstruction extends CPInstruction implements
 		return output.getName();
 	}
 
+	public CPOperand getOutput(){
+		return output;
+	}
+
 	public CPOperand[] getInputs(){
 		return new CPOperand[]{input1, input2, input3};
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
index e250bb2..46111b9 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnBuiltinCPInstruction.java
@@ -20,6 +20,7 @@
 package org.apache.sysds.runtime.instructions.cp;
 
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.sysds.common.Types.DataType;
@@ -47,6 +48,10 @@ public class MultiReturnBuiltinCPInstruction extends ComputationCPInstruction {
 		return _outputs.get(i);
 	}
 
+	public List<CPOperand> getOutputs(){
+		return _outputs;
+	}
+
 	public String[] getOutputNames(){
 		return _outputs.parallelStream().map(output -> output.getName()).toArray(String[]::new);
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
index 85fbcf8..1b605ca 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/MultiReturnParameterizedBuiltinCPInstruction.java
@@ -20,6 +20,7 @@
 package org.apache.sysds.runtime.instructions.cp;
 
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.sysds.common.Types.DataType;
@@ -48,6 +49,10 @@ public class MultiReturnParameterizedBuiltinCPInstruction extends ComputationCPI
 		return _outputs.get(i);
 	}
 
+	public List<CPOperand> getOutputs(){
+		return _outputs;
+	}
+
 	public String[] getOutputNames() {
 		return _outputs.stream().map(output -> output.getName()).toArray(String[]::new);
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/SqlCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/SqlCPInstruction.java
index 4a10ab0..add9bb7 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/SqlCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/SqlCPInstruction.java
@@ -132,4 +132,8 @@ public class SqlCPInstruction extends CPInstruction {
 	public String getOutputVariableName(){
 		return _output.getName();
 	}
+
+	public CPOperand getOutput(){
+		return _output;
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/fed/FEDInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/fed/FEDInstruction.java
index 292702e..1550bc6 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/fed/FEDInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/fed/FEDInstruction.java
@@ -40,7 +40,6 @@ public abstract class FEDInstruction extends Instruction {
 	}
 	
 	protected final FEDType _fedType;
-	protected final Operator _optr;
 	protected long _tid = -1; //main
 	
 	protected FEDInstruction(FEDType type, String opcode, String istr) {
@@ -48,8 +47,8 @@ public abstract class FEDInstruction extends Instruction {
 	}
 	
 	protected FEDInstruction(FEDType type, Operator op, String opcode, String istr) {
+		super(op);
 		_fedType = type;
-		_optr = op;
 		instString = istr;
 		instOpcode = opcode;
 	}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/gpu/GPUInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/gpu/GPUInstruction.java
index 2a63b8e..e6f22e3 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/gpu/GPUInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/gpu/GPUInstruction.java
@@ -148,22 +148,17 @@ public abstract class GPUInstruction extends Instruction {
 	public final static String MISC_TIMER_CUMULATIVE_SUMPROD_KERNEL =  	   "cumSumProdk"; // time spent in cumulative sum-product cuda kernel
 
 	protected GPUINSTRUCTION_TYPE _gputype;
-	protected Operator _optr;
 
 	protected boolean _requiresLabelUpdate = false;
 
-	private GPUInstruction(String opcode, String istr) {
+	protected GPUInstruction(Operator op, String opcode, String istr) {
+		super(op);
 		instString = istr;
 
 		// prepare opcode and update requirement for repeated usage
 		instOpcode = opcode;
 		_requiresLabelUpdate = super.requiresLabelUpdate();
 	}
-
-	protected GPUInstruction(Operator op, String opcode, String istr) {
-		this(opcode, istr);
-		_optr = op;
-	}
 	
 	@Override
 	public IType getType() {
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/SPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/SPInstruction.java
index ab74e3e..9cb2a1e 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/spark/SPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/SPInstruction.java
@@ -41,7 +41,6 @@ public abstract class SPInstruction extends Instruction {
 	}
 
 	protected final SPType _sptype;
-	protected final Operator _optr;
 	protected final boolean _requiresLabelUpdate;
 
 	protected SPInstruction(SPType type, String opcode, String istr) {
@@ -49,8 +48,8 @@ public abstract class SPInstruction extends Instruction {
 	}
 
 	protected SPInstruction(SPType type, Operator op, String opcode, String istr) {
+		super(op);
 		_sptype = type;
-		_optr = op;
 		instString = istr;
 
 		// prepare opcode and update requirement for repeated usage
@@ -62,10 +61,6 @@ public abstract class SPInstruction extends Instruction {
 	public IType getType() {
 		return IType.SPARK;
 	}
-	
-	public Operator getOperator() {
-		return _optr;
-	}
 
 	public SPType getSPInstructionType() {
 		return _sptype;
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/PrivacyConstraint.java b/src/main/java/org/apache/sysds/runtime/privacy/PrivacyConstraint.java
index 45b12be..b19d981 100644
--- a/src/main/java/org/apache/sysds/runtime/privacy/PrivacyConstraint.java
+++ b/src/main/java/org/apache/sysds/runtime/privacy/PrivacyConstraint.java
@@ -19,10 +19,29 @@
 
 package org.apache.sysds.runtime.privacy;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+import org.apache.sysds.parser.DataExpression;
+import org.apache.sysds.runtime.privacy.finegrained.DataRange;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacy;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacyList;
+import org.apache.wink.json4j.JSONArray;
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+import org.apache.wink.json4j.OrderedJSONObject;
+
 /**
- * PrivacyConstraint holds all privacy constraints for data in the system at compile time and runtime. 
+ * PrivacyConstraint holds all privacy constraints for data in the system at
+ * compile time and runtime.
  */
-public class PrivacyConstraint
+public class PrivacyConstraint implements Externalizable
 {
 	public enum PrivacyLevel {
 		None,               // No data exchange constraints. Data can be shared with anyone.
@@ -31,10 +50,32 @@ public class PrivacyConstraint
 	}
 
 	protected PrivacyLevel privacyLevel = PrivacyLevel.None;
+	protected FineGrainedPrivacy fineGrainedPrivacy;
+	
+	/**
+	 * Basic Constructor with a fine-grained collection 
+	 * based on a list implementation.
+	 */
+	public PrivacyConstraint(){
+		this(new FineGrainedPrivacyList());
+	}
 
-	public PrivacyConstraint(){}
+	/**
+	 * Constructor with the option to choose between 
+	 * different fine-grained collection implementations.
+	 * @param fineGrainedPrivacyCollection the instance in which fine-grained constraints are stored
+	 */
+	public PrivacyConstraint(FineGrainedPrivacy fineGrainedPrivacyCollection){
+		setFineGrainedPrivacyConstraints(fineGrainedPrivacyCollection);
+	}
 
+	/**
+	 * Constructor with default fine-grained collection implementation
+	 * where the entire data object is set to the given privacy level.
+	 * @param privacyLevel for the entire data object.
+	 */
 	public PrivacyConstraint(PrivacyLevel privacyLevel) {
+		this();
 		setPrivacyLevel(privacyLevel);
 	}
 
@@ -45,4 +86,171 @@ public class PrivacyConstraint
 	public PrivacyLevel getPrivacyLevel(){
 		return privacyLevel;
 	}
+
+	/**
+	 * Checks if fine-grained privacy is set for this privacy constraint. 
+	 * @return true if the privacy constraint has fine-grained constraints.
+	 */
+	public boolean hasFineGrainedConstraints(){
+		return fineGrainedPrivacy.hasConstraints();
+	}
+
+	/**
+	 * Sets fine-grained privacy for the privacy constraint. 
+	 * Existing fine-grained privacy collection will be overwritten.
+	 * @param fineGrainedPrivacy fine-grained privacy instance which is set for the privacy constraint
+	 */
+	public void setFineGrainedPrivacyConstraints(FineGrainedPrivacy fineGrainedPrivacy){
+		this.fineGrainedPrivacy = fineGrainedPrivacy;
+	}
+
+	/**
+	 * Get fine-grained privacy instance. 
+	 * @return fine-grained privacy instance
+	 */
+	public FineGrainedPrivacy getFineGrainedPrivacy(){
+		return fineGrainedPrivacy;
+	}
+
+	/**
+	 * Return true if any of the elements has privacy level private
+	 * @return true if any element has privacy level private
+	 */
+	public boolean hasPrivateElements(){
+		if (privacyLevel == PrivacyLevel.Private) return true;
+		if ( hasFineGrainedConstraints() ){
+			DataRange[] dataRanges = fineGrainedPrivacy.getDataRangesOfPrivacyLevel(PrivacyLevel.Private);
+			return dataRanges != null && dataRanges.length > 0;
+		} else return false;
+	}
+
+	/**
+	 * Return true if any constraints have level Private or PrivateAggregate.
+	 * @return true if any constraints have level Private or PrivateAggregate
+	 */
+	public boolean hasConstraints(){
+		if ( privacyLevel != null && 
+			(privacyLevel == PrivacyLevel.Private || privacyLevel == PrivacyLevel.PrivateAggregation) )
+			return true;
+		else if ( hasFineGrainedConstraints() ){
+			DataRange[] privateRanges = fineGrainedPrivacy.getDataRangesOfPrivacyLevel(PrivacyLevel.Private);
+			DataRange[] aggregateRanges = fineGrainedPrivacy.getDataRangesOfPrivacyLevel(PrivacyLevel.PrivateAggregation);
+			return (privateRanges != null && privateRanges.length > 0) 
+				|| (aggregateRanges != null && aggregateRanges.length > 0);
+		} else return false;
+	}
+
+	/**
+	 * Get privacy constraints and put them into JSON object. 
+	 * @param json JSON object in which the privacy constraints are put
+	 * @return JSON object including the privacy constraints
+	 * @throws JSONException in case of errors in creating JSON object
+	 */
+	public JSONObject toJson(JSONObject json) throws JSONException {
+		if ( getPrivacyLevel() != null && getPrivacyLevel() != PrivacyLevel.None )
+			json.put(DataExpression.PRIVACY, getPrivacyLevel().name());
+		if ( hasFineGrainedConstraints() ) {
+			DataRange[] privateRanges = getFineGrainedPrivacy().getDataRangesOfPrivacyLevel(PrivacyLevel.Private);
+			JSONArray privateRangesJson = getJsonArray(privateRanges);
+			
+			DataRange[] aggregateRanges = getFineGrainedPrivacy().getDataRangesOfPrivacyLevel(PrivacyLevel.PrivateAggregation);
+			JSONArray aggregateRangesJson = getJsonArray(aggregateRanges);
+			
+			OrderedJSONObject rangesJson = new OrderedJSONObject();
+			rangesJson.put(PrivacyLevel.Private.name(), privateRangesJson);
+			rangesJson.put(PrivacyLevel.PrivateAggregation.name(), aggregateRangesJson);
+			json.put(DataExpression.FINE_GRAINED_PRIVACY, rangesJson);
+		}
+		return json;
+	}
+
+	private static JSONArray getJsonArray(DataRange[] ranges) throws JSONException {
+		JSONArray rangeObjects = new JSONArray();
+		for ( DataRange range : ranges ){
+			List<Long> rangeBegin = Arrays.stream(range.getBeginDims()).boxed().collect(Collectors.toList());
+			List<Long> rangeEnd = Arrays.stream(range.getEndDims()).boxed().collect(Collectors.toList());
+			JSONArray beginJson = new JSONArray(rangeBegin);
+			JSONArray endJson = new JSONArray(rangeEnd);
+			JSONArray rangeObject = new JSONArray();
+			rangeObject.put(beginJson);
+			rangeObject.put(endJson);
+			rangeObjects.add(rangeObject);
+		}
+		return rangeObjects;
+	}
+
+	@Override
+	public void readExternal(ObjectInput is) throws IOException {
+		this.privacyLevel = PrivacyLevel.values()[is.readInt()];
+		int fineGrainedConstraintLength = is.readInt();
+		if ( fineGrainedConstraintLength > 0 ){
+			for (int i = 0; i < fineGrainedConstraintLength; i++){
+				Integer levelIndex = (Integer) is.readInt();
+				PrivacyLevel rangePrivacy = PrivacyLevel.values()[levelIndex];
+				DataRange dataRange = readExternalDataRangeObject(is);
+				fineGrainedPrivacy.put(dataRange, rangePrivacy);
+			}
+		}
+	}
+
+	@Override
+	public void writeExternal(ObjectOutput objectOutput) throws IOException {
+		objectOutput.writeInt(getPrivacyLevel().ordinal());
+		
+		if (fineGrainedPrivacy != null && fineGrainedPrivacy.hasConstraints()){
+			List<Entry<DataRange,PrivacyLevel>> finegrainedConstraints = fineGrainedPrivacy.getAllConstraintsList();
+			objectOutput.writeInt(finegrainedConstraints.size());
+			for ( Entry<DataRange,PrivacyLevel> constraint : finegrainedConstraints ) {
+				objectOutput.writeInt(constraint.getValue().ordinal());
+				DataRange dataRange = constraint.getKey();
+				objectOutput.writeInt(dataRange.getBeginDims().length);
+				writeExternalRangeDim(objectOutput, dataRange.getBeginDims());
+				writeExternalRangeDim(objectOutput, dataRange.getEndDims());
+			}
+		}
+		else {
+			objectOutput.writeInt(0);
+		}
+	}
+
+	/**
+	 * Reads a DataRange from ObjectInput. 
+	 * @param is ObjectInput from which the DataRange is read
+	 * @return DataRange from ObjectInput
+	 * @throws IOException
+	 */
+	private static DataRange readExternalDataRangeObject(ObjectInput is) throws IOException {
+		int dimLength = is.readInt();
+		long[] beginDims = readExternalDataRangeDim(is, dimLength);
+		long[] endDims = readExternalDataRangeDim(is, dimLength);
+		return new DataRange(beginDims, endDims);
+	}
+
+	/**
+	 * Read a long array of the specified length from object input. 
+	 * @param is ObjectInput from which the long array is read
+	 * @param dimLength length of input long array
+	 * @return the input array as a long array
+	 * @throws IOException
+	 */
+	private static long[] readExternalDataRangeDim(ObjectInput is, int dimLength) throws IOException {
+		long[] dims = new long[dimLength];
+		for(int i = 0; i < dimLength; i++){
+			dims[i] = is.readLong();
+		}
+		return dims;
+	}
+
+	/**
+	 * Write the long array to ObjectOutput.
+	 * @param objectOutput ObjectOutput in which the long array is written.
+	 * @param rangeDim long array to write in ObjectOutput. 
+	 * @throws IOException
+	 */
+	private static void writeExternalRangeDim(ObjectOutput objectOutput, long[] rangeDim) throws IOException {
+		for ( long beginIndex : rangeDim ){
+			objectOutput.writeLong(beginIndex);
+		}
+	}
+
 }
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/PrivacyPropagator.java b/src/main/java/org/apache/sysds/runtime/privacy/PrivacyPropagator.java
index d1a50dd..734eb0c 100644
--- a/src/main/java/org/apache/sysds/runtime/privacy/PrivacyPropagator.java
+++ b/src/main/java/org/apache/sysds/runtime/privacy/PrivacyPropagator.java
@@ -19,15 +19,19 @@
 
 package org.apache.sysds.runtime.privacy;
 
+import java.util.*;
+
 import org.apache.sysds.parser.DataExpression;
-import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
 import org.apache.sysds.runtime.instructions.Instruction;
+import org.apache.sysds.runtime.instructions.cp.AggregateBinaryCPInstruction;
+import org.apache.sysds.runtime.instructions.cp.AggregateUnaryCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.BinaryCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.BuiltinNaryCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.CPInstruction;
 import org.apache.sysds.runtime.instructions.cp.CPOperand;
 import org.apache.sysds.runtime.instructions.cp.ComputationCPInstruction;
+import org.apache.sysds.runtime.instructions.cp.CovarianceCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.Data;
 import org.apache.sysds.runtime.instructions.cp.FunctionCallCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.MultiReturnParameterizedBuiltinCPInstruction;
@@ -37,7 +41,10 @@ import org.apache.sysds.runtime.instructions.cp.QuaternaryCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.SqlCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.UnaryCPInstruction;
 import org.apache.sysds.runtime.instructions.cp.VariableCPInstruction;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
 import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+import org.apache.sysds.runtime.privacy.finegrained.DataRange;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacy;
 import org.apache.wink.json4j.JSONException;
 import org.apache.wink.json4j.JSONObject;
 
@@ -77,7 +84,7 @@ public class PrivacyPropagator
 		else if (privacyConstraint1 != null)
 			return privacyConstraint1;
 		else if (privacyConstraint2 != null)
-			return privacyConstraint2; 
+			return privacyConstraint2;
 		return null;
 	}
 
@@ -92,50 +99,228 @@ public class PrivacyPropagator
 	public static Instruction preprocessInstruction(Instruction inst, ExecutionContext ec){
 		switch ( inst.getType() ){
 			case CONTROL_PROGRAM:
-				return preprocessCPInstruction( (CPInstruction) inst, ec );
+				return preprocessCPInstructionFineGrained( (CPInstruction) inst, ec );
 			case BREAKPOINT:
 			case SPARK:
 			case GPU:
 			case FEDERATED:
 				return inst;
 			default:
-				throwExceptionIfPrivacyActivated(inst, ec);
+				throwExceptionIfPrivacyActivated(inst);
 				return inst;
 		}
 	}
 
-	public static Instruction preprocessCPInstruction(CPInstruction inst, ExecutionContext ec){
-		switch ( inst.getCPInstructionType() )
-		{
-			case Variable:
-				return preprocessVariableCPInstruction((VariableCPInstruction) inst, ec);
-			case AggregateUnary:
-			case Reorg:
-			case Unary:
-				return preprocessUnaryCPInstruction((UnaryCPInstruction) inst, ec);
+	public static Instruction preprocessCPInstructionFineGrained(CPInstruction inst, ExecutionContext ec){
+		switch ( inst.getCPInstructionType() ){
 			case AggregateBinary:
-			case Append: 
+				if ( inst instanceof AggregateBinaryCPInstruction ){
+					// This can only be a matrix multiplication and it does not count as an aggregation in terms of privacy.
+					return preprocessAggregateBinaryCPInstruction((AggregateBinaryCPInstruction)inst, ec);
+				} else if ( inst instanceof CovarianceCPInstruction ){
+					return preprocessCovarianceCPInstruction((CovarianceCPInstruction)inst, ec);
+				} else preprocessInstructionSimple(inst, ec);
+			case AggregateTernary:
+				//TODO: Support propagation of fine-grained privacy constraints
+				return preprocessTernaryCPInstruction((ComputationCPInstruction) inst, ec);
+			case AggregateUnary:
+				// Assumption: aggregates in one or several dimensions, number of dimensions may change, only certain slices of the data may be aggregated upon, elements do not change position
+				return preprocessAggregateUnaryCPInstruction((AggregateUnaryCPInstruction)inst, ec);
+			case Append:
 			case Binary:
+				// TODO: Support propagation of fine-grained privacy constraints
 				return preprocessBinaryCPInstruction((BinaryCPInstruction) inst, ec);
-			case AggregateTernary: 
-			case Ternary:
-				return preprocessTernaryCPInstruction((ComputationCPInstruction) inst, ec);
-			case Quaternary: 
-				return preprocessQuaternary((QuaternaryCPInstruction) inst, ec);
-			case BuiltinNary:
 			case Builtin:
+			case BuiltinNary:
+				//TODO: Support propagation of fine-grained privacy constraints
 				return preprocessBuiltinNary((BuiltinNaryCPInstruction) inst, ec);
+			/*case CentralMoment:
+				break;
+			case Compression:
+				break;
+			case Covariance:
+				break;
+			case Ctable:
+				break;
+			case Dnn:
+				break;
+			 */
 			case FCall:
+				//TODO: Support propagation of fine-grained privacy constraints
 				return preprocessExternal((FunctionCallCPInstruction) inst, ec);
+			/*
+			case MMChain:
+				break;
+			case MMTSJ:
+				break;
+			case MatrixIndexing:
+				break;*/
 			case MultiReturnBuiltin:
 			case MultiReturnParameterizedBuiltin:
+				// TODO: Support propagation of fine-grained privacy constraints
 				return preprocessMultiReturn((ComputationCPInstruction)inst, ec);
+			/*case PMMJ:
+				break;*/
 			case ParameterizedBuiltin:
+				// TODO: Support propagation of fine-grained privacy constraints
 				return preprocessParameterizedBuiltin((ParameterizedBuiltinCPInstruction) inst, ec);
-			case Ctable:   
+			/*case Partition:
+				break;
+			case QPick:
+				break;
+			case QSort:
+				break;*/
+			case Quaternary:
+				// TODO: Support propagation of fine-grained privacy constraints
+				return preprocessQuaternary((QuaternaryCPInstruction) inst, ec);
+			/*case Rand:
+				break;*/
+			case Reorg:
+				// TODO: Support propagation of fine-grained privacy constraints
+				return preprocessUnaryCPInstruction((UnaryCPInstruction) inst, ec);
+			/*case Reshape:
+				break;
+			case SpoofFused:
+				break;
+			case Sql:
+				break;
+			case StringInit:
+				break;*/
+			case Ternary:
+				// TODO: Support propagation of fine-grained privacy constraints
+				return preprocessTernaryCPInstruction((ComputationCPInstruction) inst, ec);
+			/*case UaggOuterChain:
+				break;*/
+			case Unary:
+				// Assumption: No aggregation, elements do not change position, no change of dimensions
+				return preprocessUnaryCPInstruction((UnaryCPInstruction) inst, ec);
+			case Variable:
+				return preprocessVariableCPInstruction((VariableCPInstruction) inst, ec);
 			default:
 				return preprocessInstructionSimple(inst, ec);
+			
+		}
+	}
+
+	/**
+	 * Throw exception if privacy constraint activated for instruction or for input to instruction.
+	 * @param inst covariance instruction
+	 * @param ec execution context
+	 * @return input instruction if privacy constraints are not activated
+	 */
+	private static Instruction preprocessCovarianceCPInstruction(CovarianceCPInstruction inst, ExecutionContext ec){
+		throwExceptionIfPrivacyActivated(inst);
+		for ( CPOperand input : inst.getInputs() ){
+			PrivacyConstraint privacyConstraint = getInputPrivacyConstraint(ec, input);
+			if ( privacyConstraint != null){
+				throw new DMLPrivacyException("Input of instruction " + inst + " has privacy constraints activated, but the constraints are not propagated during preprocessing of instruction.");
+			}
+		}
+		return inst;
+	}
+
+	private static Instruction preprocessAggregateBinaryCPInstruction(AggregateBinaryCPInstruction inst, ExecutionContext ec){
+		PrivacyConstraint privacyConstraint1 = getInputPrivacyConstraint(ec, inst.input1);
+		PrivacyConstraint privacyConstraint2 = getInputPrivacyConstraint(ec, inst.input2);
+		if ( (privacyConstraint1 != null && privacyConstraint1.hasConstraints()) 
+			|| (privacyConstraint2 != null && privacyConstraint2.hasConstraints()) ){
+				PrivacyConstraint mergedPrivacyConstraint;
+				if ( (privacyConstraint1 != null && privacyConstraint1.hasFineGrainedConstraints() ) || (privacyConstraint2 != null && privacyConstraint2.hasFineGrainedConstraints() )){
+					MatrixBlock input1 = ec.getMatrixInput(inst.input1.getName());
+					MatrixBlock input2 = ec.getMatrixInput(inst.input2.getName());
+					mergedPrivacyConstraint = matrixMultiplicationPropagation(input1, privacyConstraint1, input2, privacyConstraint2);
+				}
+				else {
+					mergedPrivacyConstraint = mergeBinary(privacyConstraint1, privacyConstraint2);
+					inst.setPrivacyConstraint(mergedPrivacyConstraint);
+				}
+				inst.output.setPrivacyConstraint(mergedPrivacyConstraint);
+		}
+		return inst;
+	}
+
+	public static Instruction preprocessBinaryCPInstruction(BinaryCPInstruction inst, ExecutionContext ec){
+		PrivacyConstraint privacyConstraint1 = getInputPrivacyConstraint(ec, inst.input1);
+		PrivacyConstraint privacyConstraint2 = getInputPrivacyConstraint(ec, inst.input2);
+		if ( privacyConstraint1 != null || privacyConstraint2 != null) {
+			PrivacyConstraint mergedPrivacyConstraint = mergeBinary(privacyConstraint1, privacyConstraint2);
+			inst.setPrivacyConstraint(mergedPrivacyConstraint);
+			inst.output.setPrivacyConstraint(mergedPrivacyConstraint);
+		}
+		return inst;
+	}
+
+	/**
+	 * Return the merged fine-grained privacy constraint of a matrix multiplication with the given privacy constraints.
+	 * The current implementation has a tendency to create small ranges of privacy level private. These ranges could be merged
+	 * to create fewer ranges spanning the same elements.
+	 * @param input1 first input matrix block
+	 * @param privacyConstraint1 privacy constraint of the first matrix
+	 * @param input2 second input matrix block
+	 * @param privacyConstraint2 privacy constraint of the second matrix
+	 * @return merged privacy constraint
+	 */
+	public static PrivacyConstraint matrixMultiplicationPropagation(MatrixBlock input1, PrivacyConstraint privacyConstraint1, MatrixBlock input2, PrivacyConstraint privacyConstraint2){
+		// If the overall privacy level is private, then the fine-grained constraints do not have to be checked.
+		if ( (privacyConstraint1 != null && privacyConstraint1.getPrivacyLevel() == PrivacyLevel.Private) || (privacyConstraint2 != null && privacyConstraint2.getPrivacyLevel() == PrivacyLevel.Private) )
+			return new PrivacyConstraint(PrivacyLevel.Private);
+		
+		boolean hasOverallConstraintAggregate = ( (privacyConstraint1 != null && privacyConstraint1.getPrivacyLevel() == PrivacyLevel.PrivateAggregation ) || ( privacyConstraint2 != null && privacyConstraint2.getPrivacyLevel() == PrivacyLevel.PrivateAggregation));
+		PrivacyConstraint mergedConstraint = new PrivacyConstraint();
+		if ( hasOverallConstraintAggregate )
+			mergedConstraint.setPrivacyLevel(PrivacyLevel.PrivateAggregation);
+
+		int r1 = input1.getNumRows();
+		int c1 = input1.getNumColumns();
+		int r2 = input2.getNumRows();
+		int c2 = input2.getNumColumns();
+		FineGrainedPrivacy mergedFineGrainedConstraints = mergedConstraint.getFineGrainedPrivacy();
+
+		for (int i = 0; i < r1; i++){
+
+			// Get privacy of first matrix row
+			long[] beginRange1 = new long[]{i,0};
+			long[] endRange1 = new long[]{i,c1};
+			Map<DataRange, PrivacyLevel> privacyInRow = (privacyConstraint1 != null) ? privacyConstraint1.getFineGrainedPrivacy().getPrivacyLevel(new DataRange(beginRange1, endRange1)) : new HashMap<>();
+			
+			for (int j = 0; j < c2; j++){
+				// Get Privacy of Second matrix col
+				long[] beginRange2 = new long[]{0,j};
+				long[] endRange2 = new long[]{r2,j};
+				Map<DataRange, PrivacyLevel> privacyInCol = (privacyConstraint2 != null) ? privacyConstraint2.getFineGrainedPrivacy().getPrivacyLevel(new DataRange(beginRange2, endRange2)) : new HashMap<>();
+			
+				// if any elements in the row or col has privacy level private or privateaggregate, 
+				// then output element in the index should be same level.
+				long[] beginRangeMerged = new long[]{i,j};
+				long[] endRangeMerged = new long[]{i,j};
+				if ( privacyInRow.containsValue(PrivacyLevel.Private) || privacyInCol.containsValue(PrivacyLevel.Private))
+					mergedFineGrainedConstraints.put(new DataRange(beginRangeMerged, endRangeMerged), PrivacyLevel.Private);
+				else if ( !hasOverallConstraintAggregate && (privacyInRow.containsValue(PrivacyLevel.PrivateAggregation) || privacyInCol.containsValue(PrivacyLevel.PrivateAggregation) ))
+					mergedFineGrainedConstraints.put(new DataRange(beginRangeMerged, endRangeMerged), PrivacyLevel.PrivateAggregation);
+			}
+		}
+		return mergedConstraint;
+	}
+
+	/**
+	 * Propagate privacy constraint to output if any of the elements are private.
+	 * Privacy constraint is always propagated to instruction.
+	 * @param inst aggregate instruction
+	 * @param ec execution context
+	 * @return updated instruction with propagated privacy constraints
+	 */
+	private static Instruction preprocessAggregateUnaryCPInstruction(AggregateUnaryCPInstruction inst, ExecutionContext ec){
+		PrivacyConstraint privacyConstraint = getInputPrivacyConstraint(ec, inst.input1);
+		if ( privacyConstraint != null ) {
+			inst.setPrivacyConstraint(privacyConstraint);
+			if ( inst.output != null){
+				//Only propagate to output if any of the elements are private. 
+				//It is an aggregation, hence the constraint can be removed in case of any other privacy level.
+				if(privacyConstraint.hasPrivateElements())
+					inst.output.setPrivacyConstraint(new PrivacyConstraint(PrivacyLevel.Private));
+			}
 		}
+		return inst;
 	}
 
 	/**
@@ -145,7 +330,7 @@ public class PrivacyPropagator
 	 * @return instruction
 	 */
 	public static Instruction preprocessInstructionSimple(Instruction inst, ExecutionContext ec){
-		throwExceptionIfPrivacyActivated(inst, ec);
+		throwExceptionIfPrivacyActivated(inst);
 		return inst;
 	}
 
@@ -160,15 +345,12 @@ public class PrivacyPropagator
 	}
 
 	public static Instruction preprocessMultiReturn(ComputationCPInstruction inst, ExecutionContext ec){
-		if ( inst instanceof MultiReturnBuiltinCPInstruction )
-			return mergePrivacyConstraintsFromInput(inst, ec, inst.getInputs(), ((MultiReturnBuiltinCPInstruction) inst).getOutputNames() );
-		else if ( inst instanceof MultiReturnParameterizedBuiltinCPInstruction )
-			return mergePrivacyConstraintsFromInput(inst, ec, inst.getInputs(), ((MultiReturnParameterizedBuiltinCPInstruction) inst).getOutputNames() );
-		else throw new DMLRuntimeException("ComputationCPInstruction not recognized as either MultiReturnBuiltinCPInstruction or MultiReturnParameterizedBuiltinCPInstruction");
+		List<CPOperand> outputs = getOutputOperands(inst);
+		return mergePrivacyConstraintsFromInput(inst, ec, inst.getInputs(), outputs);
 	}
 
 	public static Instruction preprocessParameterizedBuiltin(ParameterizedBuiltinCPInstruction inst, ExecutionContext ec){
-		return mergePrivacyConstraintsFromInput(inst, ec, inst.getInputs(), new String[]{inst.getOutputVariableName()} );
+		return mergePrivacyConstraintsFromInput(inst, ec, inst.getInputs(), inst.getOutput() );
 	}
 
 	private static Instruction mergePrivacyConstraintsFromInput(Instruction inst, ExecutionContext ec, CPOperand[] inputs, String[] outputNames){
@@ -187,8 +369,23 @@ public class PrivacyPropagator
 	}
 
 	private static Instruction mergePrivacyConstraintsFromInput(Instruction inst, ExecutionContext ec, CPOperand[] inputs, CPOperand output){
-		String outputName = (output != null) ? output.getName() : null;
-		return mergePrivacyConstraintsFromInput(inst, ec, inputs, new String[]{outputName});	
+		return mergePrivacyConstraintsFromInput(inst, ec, inputs, getSingletonList(output));
+	}
+
+	private static Instruction mergePrivacyConstraintsFromInput(Instruction inst, ExecutionContext ec, CPOperand[] inputs, List<CPOperand> outputs){
+		if ( inputs != null && inputs.length > 0 ){
+			PrivacyConstraint[] privacyConstraints = getInputPrivacyConstraints(ec, inputs);
+			if ( privacyConstraints != null ){
+				PrivacyConstraint mergedPrivacyConstraint = mergeNary(privacyConstraints);
+				inst.setPrivacyConstraint(mergedPrivacyConstraint);
+				for ( CPOperand output : outputs ){
+					if ( output != null ) {
+						output.setPrivacyConstraint(mergedPrivacyConstraint);
+					}
+				}
+			}
+		}
+		return inst;
 	}
 
 	public static Instruction preprocessBuiltinNary(BuiltinNaryCPInstruction inst, ExecutionContext ec){
@@ -208,17 +405,6 @@ public class PrivacyPropagator
 		return mergePrivacyConstraintsFromInput(inst, ec, inst.getInputs(), inst.output);
 	}
 
-	public static Instruction preprocessBinaryCPInstruction(BinaryCPInstruction inst, ExecutionContext ec){
-		PrivacyConstraint privacyConstraint1 = getInputPrivacyConstraint(ec, inst.input1);
-		PrivacyConstraint privacyConstraint2 = getInputPrivacyConstraint(ec, inst.input2);
-		if ( privacyConstraint1 != null || privacyConstraint2 != null) {
-			PrivacyConstraint mergedPrivacyConstraint = mergeBinary(privacyConstraint1, privacyConstraint2);
-			inst.setPrivacyConstraint(mergedPrivacyConstraint);
-			setOutputPrivacyConstraint(ec, mergedPrivacyConstraint, inst.output);
-		}
-		return inst;
-	}
-
 	public static Instruction preprocessUnaryCPInstruction(UnaryCPInstruction inst, ExecutionContext ec){
 		return propagateInputPrivacy(inst, ec, inst.input1, inst.output);
 	}
@@ -228,38 +414,32 @@ public class PrivacyPropagator
 			case CreateVariable:
 				return propagateSecondInputPrivacy(inst, ec);
 			case AssignVariable:
-				//Assigns scalar, hence it does not have privacy activated
-				return inst;
+				return propagateInputPrivacy(inst, ec, inst.getInput1(), inst.getInput2());
 			case CopyVariable:
 			case MoveVariable:
-				return propagateFirstInputPrivacy(inst, ec);
-			case RemoveVariable:
-				return propagateAllInputPrivacy(inst, ec);
 			case RemoveVariableAndFile:
-				return propagateFirstInputPrivacy(inst, ec);
-			case CastAsScalarVariable: 
-				return propagateCastAsScalarVariablePrivacy(inst, ec);
 			case CastAsMatrixVariable:
 			case CastAsFrameVariable:
+			case Write:
+			case SetFileName:
 				return propagateFirstInputPrivacy(inst, ec);
+			case RemoveVariable:
+				return propagateAllInputPrivacy(inst, ec);
+			case CastAsScalarVariable:
 			case CastAsDoubleVariable:
 			case CastAsIntegerVariable:
 			case CastAsBooleanVariable:
 				return propagateCastAsScalarVariablePrivacy(inst, ec);
 			case Read:
 				return inst;
-			case Write:
-				return propagateFirstInputPrivacy(inst, ec);
-			case SetFileName:
-				return propagateFirstInputPrivacy(inst, ec);
 			default:
-				throwExceptionIfPrivacyActivated(inst, ec);
+				throwExceptionIfPrivacyActivated(inst);
 				return inst;
 		}
 	}
 
-	private static void throwExceptionIfPrivacyActivated(Instruction inst, ExecutionContext ec){
-		if ( inst.getPrivacyConstraint() != null && inst.getPrivacyConstraint().getPrivacyLevel() == PrivacyLevel.Private ) {
+	private static void throwExceptionIfPrivacyActivated(Instruction inst){
+		if ( inst.getPrivacyConstraint() != null && inst.getPrivacyConstraint().hasConstraints() ) {
 			throw new DMLPrivacyException("Instruction " + inst + " has privacy constraints activated, but the constraints are not propagated during preprocessing of instruction.");
 		}
 	}
@@ -325,7 +505,7 @@ public class PrivacyPropagator
 		if ( privacyConstraint != null ) {
 			inst.setPrivacyConstraint(privacyConstraint);
 			if ( outputOperand != null)
-				setOutputPrivacyConstraint(ec, privacyConstraint, outputOperand);
+				outputOperand.setPrivacyConstraint(privacyConstraint);
 		}
 		return inst;
 	}
@@ -361,10 +541,6 @@ public class PrivacyPropagator
 		return null;
 	}
 
-	private static void setOutputPrivacyConstraint(ExecutionContext ec, PrivacyConstraint privacyConstraint, CPOperand output){
-		setOutputPrivacyConstraint(ec, privacyConstraint, output.getName());
-	}
-
 	/**
 	 * Set privacy constraint of data variable with outputName 
 	 * if the variable exists and the privacy constraint is not null.
@@ -383,13 +559,14 @@ public class PrivacyPropagator
 	}
 
 	public static void postProcessInstruction(Instruction inst, ExecutionContext ec){
-		PrivacyConstraint instructionPrivacyConstraint = inst.getPrivacyConstraint();
-		if ( privacyConstraintActivated(instructionPrivacyConstraint) )
-		{
-			String[] instructionOutputNames = getOutputVariableName(inst);
-			if ( instructionOutputNames != null && instructionOutputNames.length > 0 )
-				for ( String instructionOutputName : instructionOutputNames )
-					setOutputPrivacyConstraint(ec, instructionPrivacyConstraint, instructionOutputName);
+		// if inst has output
+		List<CPOperand> instOutputs = getOutputOperands(inst);
+		if (!instOutputs.isEmpty()){
+			for ( CPOperand output : instOutputs ){
+				PrivacyConstraint outputPrivacyConstraint = output.getPrivacyConstraint();
+				if ( privacyConstraintActivated(outputPrivacyConstraint) )
+					setOutputPrivacyConstraint(ec, outputPrivacyConstraint, output.getName());
+			}
 		}
 	}
 
@@ -399,6 +576,7 @@ public class PrivacyPropagator
 			|| instructionPrivacyConstraint.privacyLevel == PrivacyLevel.PrivateAggregation);
 	}
 
+	@SuppressWarnings("unused")
 	private static String[] getOutputVariableName(Instruction inst){
 		String[] instructionOutputNames = null;
 		// The order of the following statements is important
@@ -414,4 +592,25 @@ public class PrivacyPropagator
 			instructionOutputNames = new String[]{((SqlCPInstruction) inst).getOutputVariableName()};
 		return instructionOutputNames;
 	}
+
+	private static List<CPOperand> getOutputOperands(Instruction inst){
+		// The order of the following statements is important
+		if ( inst instanceof MultiReturnParameterizedBuiltinCPInstruction )
+			return ((MultiReturnParameterizedBuiltinCPInstruction) inst).getOutputs();
+		else if ( inst instanceof MultiReturnBuiltinCPInstruction )
+			return ((MultiReturnBuiltinCPInstruction) inst).getOutputs();
+		else if ( inst instanceof ComputationCPInstruction )
+			return getSingletonList(((ComputationCPInstruction) inst).getOutput());
+		else if ( inst instanceof VariableCPInstruction )
+			return getSingletonList(((VariableCPInstruction) inst).getOutput());
+		else if ( inst instanceof SqlCPInstruction )
+			return getSingletonList(((SqlCPInstruction) inst).getOutput());
+		return new ArrayList<>();
+	}
+
+	private static List<CPOperand> getSingletonList(CPOperand operand){
+		if ( operand != null)
+			return new ArrayList<>(Collections.singletonList(operand));
+		return new ArrayList<>();
+	}
 }
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/PrivacyUtils.java b/src/main/java/org/apache/sysds/runtime/privacy/PrivacyUtils.java
new file mode 100644
index 0000000..15fff32
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/privacy/PrivacyUtils.java
@@ -0,0 +1,79 @@
+/*
+ * 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.runtime.privacy;
+
+import org.apache.sysds.api.DMLException;
+import org.apache.sysds.parser.Expression;
+import org.apache.sysds.parser.StringIdentifier;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+import org.apache.sysds.runtime.privacy.finegrained.DataRange;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacy;
+import org.apache.wink.json4j.JSON;
+import org.apache.wink.json4j.JSONArray;
+import org.apache.wink.json4j.JSONArtifact;
+import org.apache.wink.json4j.JSONException;
+import org.apache.wink.json4j.JSONObject;
+
+public class PrivacyUtils {
+
+	public static void setFineGrainedPrivacy(PrivacyConstraint privacyConstraint, Expression eFineGrainedPrivacy){
+		FineGrainedPrivacy fineGrainedPrivacy = privacyConstraint.getFineGrainedPrivacy();
+		StringIdentifier fgPrivacyIdentifier = (StringIdentifier) eFineGrainedPrivacy;
+		String fgPrivacyValue = fgPrivacyIdentifier.getValue();
+		try {
+			putFineGrainedConstraintsFromString(fineGrainedPrivacy, fgPrivacyValue);
+		} catch (JSONException exception){
+			throw new DMLException("JSONException: " + exception);
+		}
+		privacyConstraint.setFineGrainedPrivacyConstraints(fineGrainedPrivacy);
+	}
+
+	private static void putFineGrainedConstraintsFromString(FineGrainedPrivacy fineGrainedPrivacy, String fgPrivacyValue)
+		throws JSONException {
+		JSONArtifact fgPrivacyJson = JSON.parse(fgPrivacyValue);
+		JSONObject fgPrivacyObject = (JSONObject)fgPrivacyJson;
+		JSONArray keys = fgPrivacyObject.names();
+		for ( int i = 0; i < keys.length(); i++ ){
+			String key = keys.getString(i);
+			putFineGrainedConstraint(fgPrivacyObject, fineGrainedPrivacy, key);
+		}
+	}
+
+	private static void putFineGrainedConstraint(JSONObject fgPrivacyObject, FineGrainedPrivacy fineGrainedPrivacy, String key)
+		throws JSONException {
+		JSONArray privateArray = fgPrivacyObject.getJSONArray(key);
+		for (Object range : privateArray.toArray()){
+			DataRange dataRange = getDataRangeFromObject(range);
+			fineGrainedPrivacy.put(dataRange, PrivacyLevel.valueOf(key));
+		}
+	}
+
+	private static DataRange getDataRangeFromObject(Object range) throws JSONException {
+		JSONArray beginDims = ((JSONArray)range).getJSONArray(0);
+		JSONArray endDims = ((JSONArray)range).getJSONArray(1);
+		long[] beginDimsLong = new long[beginDims.length()];
+		long[] endDimsLong = new long[endDims.length()];
+		for ( int dimIndex = 0; dimIndex < beginDims.length(); dimIndex++ ){
+			beginDimsLong[dimIndex] = beginDims.getLong(dimIndex);
+			endDimsLong[dimIndex] = endDims.getLong(dimIndex);
+		}
+		return new DataRange(beginDimsLong, endDimsLong);
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/finegrained/DataRange.java b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/DataRange.java
new file mode 100644
index 0000000..6e77e34
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/DataRange.java
@@ -0,0 +1,114 @@
+/*
+ * 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.runtime.privacy.finegrained;
+
+import java.util.Arrays;
+
+/**
+ * A DataRange instance marks a part of a CachableData data object.
+ * The beginDims marks the beginning for all dimensions and
+ * the endDims marks the end for all dimensions. 
+ * DataRange is very similar to org.apache.sysds.runtime.util.IndexRange, 
+ * except that DataRange supports more than two dimensions. 
+ */
+public class DataRange {
+
+	private long[] _beginDims;
+	private long[] _endDims;
+
+	public DataRange(long[] beginDims, long[] endDims){
+		_beginDims = beginDims;
+		_endDims = endDims;
+	}
+
+	public long[] getBeginDims(){
+		return _beginDims;
+	}
+
+	public long[] getEndDims(){
+		return _endDims;
+	}
+
+	/**
+	 * Returns true if this data range overlaps with the given data range. 
+	 * An overlap means that the data ranges have some overlap in all dimension. 
+	 * @param dataRange for which the overlap is checked
+	 * @return true if the data ranges overlap or false if not
+	 */
+	public boolean overlaps(DataRange dataRange){
+		long[] dataRangeBegin = dataRange.getBeginDims();
+		long[] dataRangeEnd = dataRange.getEndDims();
+
+		if (_beginDims.length != dataRangeBegin.length 
+			|| _endDims.length != dataRangeEnd.length)
+		{
+			return false;
+		}
+
+		for ( int i = 0; i < _beginDims.length; i++ )
+			if ( dimensionOutOfRange(dataRangeBegin, dataRangeEnd, i) )
+				return false;
+
+		return true;
+	}
+
+	/**
+	 * Returns true if the given index is in the data range.
+	 * Being in the data range means that the index has to be in the range for all dimensions.
+	 * @param index of an element for which it is checked if it is in the range
+	 * @return true if the index is in the range and false otherwise
+	 */
+	public boolean contains(long[] index){
+		if ( _beginDims.length != index.length )
+			return false;
+		for ( int i = 0; i < _beginDims.length; i++ )
+			if ( _beginDims[i] > index[i] || _endDims[i] < index[i] )
+				return false;
+		return true;
+	}
+
+	/**
+	 * Returns true if the given DataRange is not overlapping in the given dimension
+	 * @param dataRangeBegin begin dimensions
+	 * @param dataRangeEnd end dimensions
+	 * @param i dimension
+	 * @return true if out of range
+	 */
+	private boolean dimensionOutOfRange(long[] dataRangeBegin, long[] dataRangeEnd, int i){
+		return (_beginDims[i] < dataRangeBegin[i] && _endDims[i] < dataRangeBegin[i]) 
+				|| (_beginDims[i] > dataRangeBegin[i] && _beginDims[i] > dataRangeEnd[i] );
+	}
+
+	@Override
+	public String toString(){
+		return Arrays.toString(new String[]{
+			Arrays.toString(_beginDims), Arrays.toString(_endDims)});
+	}
+
+	@Override
+	public boolean equals(Object other){
+		if ( other instanceof DataRange ) {
+			DataRange otherRange = (DataRange)other;
+			return Arrays.equals(_beginDims,otherRange.getBeginDims())
+				&& Arrays.equals(_endDims,otherRange.getEndDims());
+		}
+		return false;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacy.java b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacy.java
new file mode 100644
index 0000000..725b17c
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacy.java
@@ -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.sysds.runtime.privacy.finegrained;
+
+import java.util.ArrayList;
+import java.util.Map;
+
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+
+public interface FineGrainedPrivacy {
+
+	/**
+	 * Set privacy level of the given data range.
+	 * @param dataRange representing the range for which the privacy is set
+	 * @param privacyLevel the level of privacy for the given data range
+	 */
+	public void put(DataRange dataRange, PrivacyLevel privacyLevel);
+
+	/**
+	 * Get the data ranges and related privacy levels within given data search range.
+	 * @param searchRange the range from which all privacy levels are retrieved
+	 * @return all mappings from range to privacy level within the given search range
+	 */
+	public Map<DataRange,PrivacyLevel> getPrivacyLevel(DataRange searchRange);
+
+	/**
+	 * Get the data ranges and related privacy levels of the element with the given index.
+	 * @param searchIndex index of element
+	 * @return all mappings from range to privacy level for the given search element
+	 */
+	public Map<DataRange,PrivacyLevel> getPrivacyLevelOfElement(long[] searchIndex);
+
+	/**
+	 * Get all data ranges for the given privacy level.
+	 * @param privacyLevel for which data ranges are found
+	 * @return all data ranges with the given privacy level
+	 */
+	public DataRange[] getDataRangesOfPrivacyLevel(PrivacyLevel privacyLevel);
+
+	/**
+	 * Remove all fine-grained privacy constraints.
+	 */
+	public void removeAllConstraints();
+
+	/**
+	 * True if any fine-grained constraints has been set. 
+	 * @return true if any fine-grained constraint is set
+	 */
+	public boolean hasConstraints();
+
+	/**
+	 * Get all fine-grained constraints as a map from privacy level to 
+	 * an array of data ranges represented as two-dimensional long arrays.
+	 * @return map from privacy level to array of data ranges
+	 */
+	public Map<String, long[][][]> getAllConstraints();
+
+	/**
+	 * Return all fine-grained privacy constraints as an arraylist. 
+	 * @return all constraints
+	 */
+	public ArrayList<Map.Entry<DataRange, PrivacyLevel>> getAllConstraintsList();
+}
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacyList.java b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacyList.java
new file mode 100644
index 0000000..1f266af
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacyList.java
@@ -0,0 +1,126 @@
+/*
+ * 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.runtime.privacy.finegrained;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+
+/**
+ * Simple implementation of retrieving fine-grained privacy constraints
+ * based on pairs in an ArrayList.
+ */
+public class FineGrainedPrivacyList implements FineGrainedPrivacy {
+
+	private ArrayList<Map.Entry<DataRange, PrivacyLevel>> constraintCollection = new ArrayList<>();
+
+	@Override
+	public void put(DataRange dataRange, PrivacyLevel privacyLevel) {
+		constraintCollection.add(new AbstractMap.SimpleEntry<>(dataRange, privacyLevel));
+	}
+
+	@Override
+	public Map<DataRange,PrivacyLevel> getPrivacyLevel(DataRange searchRange) {
+		Map<DataRange, PrivacyLevel> matches = new LinkedHashMap<>();
+		for ( Map.Entry<DataRange, PrivacyLevel> constraint : constraintCollection ){
+			if ( constraint.getKey().overlaps(searchRange) ) 
+				matches.put(constraint.getKey(), constraint.getValue());
+		}
+		return matches;
+	}
+
+	@Override
+	public Map<DataRange,PrivacyLevel> getPrivacyLevelOfElement(long[] searchIndex) {
+		Map<DataRange, PrivacyLevel> matches = new LinkedHashMap<>();
+		constraintCollection.forEach( constraint -> { 
+			if (constraint.getKey().contains(searchIndex)) 
+				matches.put(constraint.getKey(), constraint.getValue()); 
+		} );
+		return matches;
+	}
+
+	@Override
+	public DataRange[] getDataRangesOfPrivacyLevel(PrivacyLevel privacyLevel) {
+		ArrayList<DataRange> matches = new ArrayList<>();
+		constraintCollection.forEach(constraint -> {
+			if (constraint.getValue() == privacyLevel) 
+				matches.add(constraint.getKey());
+		} );
+		return matches.toArray(new DataRange[0]);
+	}
+
+	@Override
+	public void removeAllConstraints() {
+		constraintCollection.clear();
+	}
+
+	@Override
+	public boolean hasConstraints() {
+		return !constraintCollection.isEmpty();
+	}
+
+	@Override
+	public Map<String, long[][][]> getAllConstraints() {
+		ArrayList<long[][]> privateRanges = new ArrayList<>();
+		ArrayList<long[][]> aggregateRanges = new ArrayList<>();
+		constraintCollection.forEach(constraint -> {
+			if ( constraint.getValue() == PrivacyLevel.Private )
+				privateRanges.add(new long[][]{constraint.getKey().getBeginDims(), constraint.getKey().getEndDims()});
+			else if ( constraint.getValue() == PrivacyLevel.PrivateAggregation )
+				aggregateRanges.add(new long[][]{constraint.getKey().getBeginDims(), constraint.getKey().getEndDims()});
+		});
+		Map<String, long[][][]> constraintMap = new HashMap<>();
+		constraintMap.put(PrivacyLevel.Private.name(), privateRanges.toArray(new long[0][][]));
+		constraintMap.put(PrivacyLevel.PrivateAggregation.name(), privateRanges.toArray(new long[0][][]));
+		return constraintMap;
+	}
+
+	@Override
+	public ArrayList<Map.Entry<DataRange, PrivacyLevel>> getAllConstraintsList() {
+		return constraintCollection;
+	}
+
+	@Override
+	public boolean equals(Object other){
+		if ( other instanceof FineGrainedPrivacyList ){
+			FineGrainedPrivacyList otherFGP = (FineGrainedPrivacyList) other;
+			if ( !otherFGP.hasConstraints() && !hasConstraints() )
+				return true;
+			if ( !otherFGP.hasConstraints() || !hasConstraints() )
+				return false;
+			return otherFGP.getAllConstraintsList().equals(constraintCollection);
+			
+		}
+		return false;
+	}
+
+	@Override
+	public String toString(){
+		StringBuilder stringBuilder = new StringBuilder();
+		for ( Map.Entry<DataRange,PrivacyLevel> entry : constraintCollection ){
+			stringBuilder.append(entry.getKey().toString() + " : " + entry.getValue().name());
+		}
+		return stringBuilder.toString();
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacyMap.java b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacyMap.java
new file mode 100644
index 0000000..c6bd848
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/privacy/finegrained/FineGrainedPrivacyMap.java
@@ -0,0 +1,105 @@
+/*
+ * 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.runtime.privacy.finegrained;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+
+/**
+ * Simple implementation of retrieving fine-grained privacy constraints based on
+ * iterating a LinkedHashMap.
+ */
+public class FineGrainedPrivacyMap implements FineGrainedPrivacy {
+
+	private Map<DataRange, PrivacyLevel> constraintCollection = new LinkedHashMap<>();
+
+	@Override
+	public void put(DataRange dataRange, PrivacyLevel privacyLevel) {
+		constraintCollection.put(dataRange, privacyLevel);
+	}
+
+	@Override
+	public Map<DataRange, PrivacyLevel> getPrivacyLevel(DataRange searchRange) {
+		Map<DataRange, PrivacyLevel> matches = new LinkedHashMap<>();
+		constraintCollection.forEach((range, level) -> {
+			if (range.overlaps(searchRange))
+				matches.put(range, level);
+		});
+		return matches;
+	}
+
+	@Override
+	public Map<DataRange, PrivacyLevel> getPrivacyLevelOfElement(long[] searchIndex) {
+		Map<DataRange, PrivacyLevel> matches = new LinkedHashMap<>();
+		constraintCollection.forEach((range, level) -> {
+			if (range.contains(searchIndex))
+				matches.put(range, level);
+		});
+		return matches;
+	}
+
+	@Override
+	public DataRange[] getDataRangesOfPrivacyLevel(PrivacyLevel privacyLevel) {
+		ArrayList<DataRange> matches = new ArrayList<>();
+		constraintCollection.forEach((k, v) -> {
+			if (v == privacyLevel)
+				matches.add(k);
+		});
+		return matches.toArray(new DataRange[0]);
+	}
+
+	@Override
+	public void removeAllConstraints() {
+		constraintCollection.clear();
+	}
+
+	@Override
+	public boolean hasConstraints() {
+		return !constraintCollection.isEmpty();
+	}
+
+	@Override
+	public Map<String, long[][][]> getAllConstraints() {
+		ArrayList<long[][]> privateRanges = new ArrayList<>();
+		ArrayList<long[][]> aggregateRanges = new ArrayList<>();
+		constraintCollection.forEach((range, privacylevel) -> {
+			if (privacylevel == PrivacyLevel.Private)
+				privateRanges.add(new long[][] { range.getBeginDims(), range.getEndDims() });
+			else if (privacylevel == PrivacyLevel.PrivateAggregation)
+				aggregateRanges.add(new long[][] { range.getBeginDims(), range.getEndDims() });
+		});
+		Map<String, long[][][]> constraintMap = new LinkedHashMap<>();
+		constraintMap.put(PrivacyLevel.Private.name(), privateRanges.toArray(new long[0][][]));
+		constraintMap.put(PrivacyLevel.PrivateAggregation.name(), privateRanges.toArray(new long[0][][]));
+		return constraintMap;
+	}
+
+	@Override
+	public ArrayList<Entry<DataRange, PrivacyLevel>> getAllConstraintsList() {
+		ArrayList<Map.Entry<DataRange, PrivacyLevel>> outputList = new ArrayList<>();
+		constraintCollection.forEach((k,v)->outputList.add(new AbstractMap.SimpleEntry<>(k,v)));
+		return outputList;
+	}
+}
diff --git a/src/main/java/org/apache/sysds/runtime/util/HDFSTool.java b/src/main/java/org/apache/sysds/runtime/util/HDFSTool.java
index af7471c..f19f1ca 100644
--- a/src/main/java/org/apache/sysds/runtime/util/HDFSTool.java
+++ b/src/main/java/org/apache/sysds/runtime/util/HDFSTool.java
@@ -456,11 +456,6 @@ public class HDFSTool
 			}
 		}
 
-		//add privacy constraints
-		if ( privacyConstraint != null ){
-			mtd.put(DataExpression.PRIVACY, privacyConstraint.getPrivacyLevel().name());
-		}
-
 		//add username and time
 		String userName = System.getProperty("user.name");
 		if (StringUtils.isNotEmpty(userName)) {
@@ -478,6 +473,11 @@ public class HDFSTool
 		SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss z");
 		mtd.put(DataExpression.CREATEDPARAM, sdf.format(new Date()));
 
+		//add privacy constraints
+		if ( privacyConstraint != null ){
+			privacyConstraint.toJson(mtd);
+		}
+
 		return mtd.toString(4); // indent with 4 spaces	
 	}
 
diff --git a/src/test/java/org/apache/sysds/test/AutomatedTestBase.java b/src/test/java/org/apache/sysds/test/AutomatedTestBase.java
index 58b9282..4e45584 100644
--- a/src/test/java/org/apache/sysds/test/AutomatedTestBase.java
+++ b/src/test/java/org/apache/sysds/test/AutomatedTestBase.java
@@ -775,7 +775,6 @@ public abstract class AutomatedTestBase {
 		}
 	}
 
-
 	public static ValueType readDMLMetaDataValueType(String fileName) {
 		try {
 			JSONObject meta = getMetaDataJSON(fileName);
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/BuiltinGLMTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/BuiltinGLMTest.java
index 5ea7c79..9de2265 100644
--- a/src/test/java/org/apache/sysds/test/functions/privacy/BuiltinGLMTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/BuiltinGLMTest.java
@@ -55,6 +55,7 @@ public class BuiltinGLMTest extends AutomatedTestBase
 
 	protected int numRecords, numFeatures, distFamilyType, linkType, intercept;
 	protected double distParam, linkPower, logFeatureVarianceDisbalance, avgLinearForm, stdevLinearForm, dispersion;
+	protected final static boolean runAll = false;
 
 	public BuiltinGLMTest(int numRecords_, int numFeatures_, int distFamilyType_, double distParam_,
 			int linkType_, double linkPower_, double logFeatureVarianceDisbalance_,
@@ -228,6 +229,9 @@ public class BuiltinGLMTest extends AutomatedTestBase
 				{  100,   10,  2,  1.0,  2,  0.0,  3.0,   0.0,  2.0,  2.5 },   // Binomial two-column.logit
 				{  200,   10,  2,  1.0,  3,  0.0,  3.0,   0.0,  2.0,  2.5 },   // Binomial two-column.probit
 		};
-		return Arrays.asList(data);
+		if ( runAll )
+			return Arrays.asList(data);
+		else
+			return Arrays.asList(new Object[][]{data[0]});
 	}
 }
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/FederatedL2SVMTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/FederatedL2SVMTest.java
index b3180cc..8aa4297 100644
--- a/src/test/java/org/apache/sysds/test/functions/privacy/FederatedL2SVMTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/FederatedL2SVMTest.java
@@ -19,9 +19,8 @@
 
 package org.apache.sysds.test.functions.privacy;
 
-import org.junit.Ignore;
+import org.apache.sysds.runtime.DMLRuntimeException;
 import org.junit.Test;
-import org.apache.sysds.api.DMLException;
 import org.apache.sysds.api.DMLScript;
 import org.apache.sysds.common.Types;
 import org.apache.sysds.runtime.meta.MatrixCharacteristics;
@@ -36,7 +35,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 @net.jcip.annotations.NotThreadSafe
-@Ignore //FIXME: fix privacy propagation for L2SVM
 public class FederatedL2SVMTest extends AutomatedTestBase {
 
 	private final static String TEST_DIR = "functions/federated/";
@@ -59,14 +57,16 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 	public void federatedL2SVMCPPrivateAggregationX1() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false,null, true, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateAggregationX2() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -82,14 +82,16 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 	public void federatedL2SVMCPPrivateFederatedX1() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateFederatedX2() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -105,42 +107,48 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 	public void federatedL2SVMCPPrivateMatrixX1() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, null, privacyConstraints, PrivacyLevel.Private, false, null, false, null);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, null, privacyConstraints, PrivacyLevel.Private,
+			false, null, false, null);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateMatrixX2() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, null, privacyConstraints, PrivacyLevel.Private, false, null, false, null);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, null, privacyConstraints, PrivacyLevel.Private,
+			false, null, false, null);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateMatrixY() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, null, privacyConstraints, PrivacyLevel.Private, false, null, false, null);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, null, privacyConstraints, PrivacyLevel.Private,
+			false, null, false, null);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateFederatedAndMatrixX1() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, privacyConstraints, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, privacyConstraints, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateFederatedAndMatrixX2() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, privacyConstraints, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, privacyConstraints, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void federatedL2SVMCPPrivateFederatedAndMatrixY() throws JSONException {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, privacyConstraints, PrivacyLevel.Private, false, null, false, null);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, privacyConstraints, PrivacyLevel.Private,
+			false, null, false, null);
 	}
 
 	// Privacy Level Private Combinations
@@ -150,7 +158,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -158,7 +167,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -166,7 +176,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -175,7 +186,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	// Privacy Level PrivateAggregation Combinations
@@ -184,7 +196,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -192,7 +205,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -200,7 +214,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -209,7 +224,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	// Privacy Level Combinations
@@ -218,7 +234,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -226,7 +243,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -234,7 +252,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -242,7 +261,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -250,7 +270,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("Y", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVMNoException(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -258,7 +279,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	// Require Federated Workers to return matrix
@@ -268,7 +290,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		rows = 1000; cols = 1;
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -276,7 +299,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		rows = 1000; cols = 1;
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.PrivateAggregation));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.PrivateAggregation,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -284,7 +308,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		rows = 1000; cols = 1;
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X1", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -292,7 +317,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		rows = 1000; cols = 1;
 		Map<String, PrivacyConstraint> privacyConstraints = new HashMap<>();
 		privacyConstraints.put("X2", new PrivacyConstraint(PrivacyLevel.Private));
-		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private, false, null, true, DMLException.class);
+		federatedL2SVM(Types.ExecMode.SINGLE_NODE, privacyConstraints, null, PrivacyLevel.Private,
+			false, null, true, DMLRuntimeException.class);
 	}
 
 	private void federatedL2SVMNoException(Types.ExecMode execMode, Map<String,
@@ -300,7 +326,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 			PrivacyLevel expectedPrivacyLevel)
 		throws JSONException
 	{
-		federatedL2SVM(execMode, privacyConstraintsFederated, privacyConstraintsMatrix, expectedPrivacyLevel, false, null, false, null);
+		federatedL2SVM(execMode, privacyConstraintsFederated, privacyConstraintsMatrix, expectedPrivacyLevel,
+			false, null, false, null);
 	}
 
 	private void federatedL2SVM(Types.ExecMode execMode, Map<String, PrivacyConstraint> privacyConstraintsFederated,
@@ -314,7 +341,7 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 		if(rtplatform == Types.ExecMode.SPARK) {
 			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
 		}
-		Process t1 = null, t2 = null;
+		Thread t1 = null, t2 = null;
 
 		try {
 			getAndLoadTestConfiguration(TEST_NAME);
@@ -355,8 +382,8 @@ public class FederatedL2SVMTest extends AutomatedTestBase {
 			fullDMLScriptName = "";
 			int port1 = getRandomAvailablePort();
 			int port2 = getRandomAvailablePort();
-			t1 = startLocalFedWorker(port1);
-			t2 = startLocalFedWorker(port2);
+			t1 = startLocalFedWorkerThread(port1);
+			t2 = startLocalFedWorkerThread(port2);
 
 			TestConfiguration config = availableTestConfigurations.get(TEST_NAME);
 			loadTestConfiguration(config);
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/FederatedWorkerHandlerTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/FederatedWorkerHandlerTest.java
index 6c7ce4b..07f18fe 100644
--- a/src/test/java/org/apache/sysds/test/functions/privacy/FederatedWorkerHandlerTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/FederatedWorkerHandlerTest.java
@@ -21,28 +21,27 @@ package org.apache.sysds.test.functions.privacy;
 
 import java.util.Arrays;
 
-import org.apache.sysds.api.DMLException;
 import org.apache.sysds.api.DMLScript;
+import org.apache.sysds.runtime.DMLRuntimeException;
 import org.apache.sysds.runtime.meta.MatrixCharacteristics;
 import org.apache.sysds.runtime.privacy.PrivacyConstraint;
 import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
 import org.apache.sysds.test.AutomatedTestBase;
 import org.apache.sysds.test.TestConfiguration;
 import org.apache.sysds.test.TestUtils;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.apache.sysds.common.Types;
 import static java.lang.Thread.sleep;
+import static org.junit.Assert.assertTrue;
 
 @net.jcip.annotations.NotThreadSafe
-@Ignore //FIXME: fix privacy propagation for various operations
 public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 
 	private static final String TEST_DIR = "functions/federated/";
 	private static final String TEST_DIR_SCALAR = TEST_DIR + "matrix_scalar/";
 	private final static String TEST_CLASS_DIR = TEST_DIR + FederatedWorkerHandlerTest.class.getSimpleName() + "/";
 	private final static String TEST_CLASS_DIR_SCALAR = TEST_DIR_SCALAR + FederatedWorkerHandlerTest.class.getSimpleName() + "/";
-	private static final String TEST_PROG_SCALAR_ADDITION_MATRIX = "ScalarAdditionFederatedMatrix";
+	private static final String TEST_PROG_SCALAR_ADDITION_MATRIX = "FederatedScalarAdditionMatrix";
 	private final static String AGGREGATION_TEST_NAME = "FederatedSumTest";
 	private final static String TRANSFER_TEST_NAME = "FederatedRCBindTest";
 	private final static String MATVECMULT_TEST_NAME = "FederatedMultiplyTest";
@@ -64,12 +63,12 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 
 	@Test
 	public void scalarPrivateTest(){
-		scalarTest(PrivacyLevel.Private, DMLException.class);
+		scalarTest(PrivacyLevel.Private, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void scalarPrivateAggregationTest(){
-		scalarTest(PrivacyLevel.PrivateAggregation, DMLException.class);
+		scalarTest(PrivacyLevel.PrivateAggregation, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -113,11 +112,11 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 			t.start();
 			sleep(FED_WORKER_WAIT);
 			fullDMLScriptName = SCRIPT_DIR + TEST_DIR_SCALAR + dmlFile + ".dml";
-			programArgs = new String[]{"-checkPrivacy", "-args",
-					TestUtils.federatedAddress(FEDERATED_WORKER_HOST, FEDERATED_WORKER_PORT, input("M")),
-					Integer.toString(rows), Integer.toString(cols),
-					Integer.toString(s),
-					output("R")};
+			programArgs = new String[]{"-checkPrivacy", "-nvargs",
+					"in=" + TestUtils.federatedAddress(FEDERATED_WORKER_HOST, FEDERATED_WORKER_PORT, input("M")),
+					"rows=" + Integer.toString(rows), "cols=" + Integer.toString(cols),
+					"scalar=" + Integer.toString(s),
+					"out=" + output("R")};
 			boolean exceptionExpected = (expectedException != null);
 			runTest(true, exceptionExpected, expectedException, -1);
 
@@ -127,7 +126,8 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 			e.printStackTrace();
 			assert (false);
 		} finally {
-			assert(checkedPrivacyConstraintsContains(privacyLevel));
+			assertTrue("The privacy level " + privacyLevel.toString() + " should have been checked during execution",
+				checkedPrivacyConstraintsContains(privacyLevel));
 			rtplatform = platformOld;
 			TestUtils.shutdownThread(t);
 			rtplatform = platformOld;
@@ -137,7 +137,7 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 
 	@Test
 	public void aggregatePrivateTest() {
-		federatedSum(Types.ExecMode.SINGLE_NODE, PrivacyLevel.Private, DMLException.class);
+		federatedSum(Types.ExecMode.SINGLE_NODE, PrivacyLevel.Private, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -161,7 +161,7 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 		double[][] A = getRandomMatrix(rows, cols, -10, 10, 1, 1);
 		writeInputMatrixWithMTD("A", A, false, new MatrixCharacteristics(rows, cols, blocksize, rows * cols), new PrivacyConstraint(privacyLevel));
 		int port = getRandomAvailablePort();
-		Process t = startLocalFedWorker(port);
+		Thread t = startLocalFedWorkerThread(port);
 
 		// we need the reference file to not be written to hdfs, so we get the correct format
 		rtplatform = Types.ExecMode.SINGLE_NODE;
@@ -188,8 +188,8 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 		TestConfiguration config = availableTestConfigurations.get("aggregation");
 		loadTestConfiguration(config);
 		fullDMLScriptName = HOME + AGGREGATION_TEST_NAME + ".dml";
-		programArgs = new String[] {"-checkPrivacy", "-args", "\"localhost:" + port + "/" + input("A") + "\"", Integer.toString(rows),
-			Integer.toString(cols), Integer.toString(rows * 2), output("S"), output("R"), output("C")};
+		programArgs = new String[] {"-checkPrivacy", "-nvargs", "in=" + TestUtils.federatedAddress(port, input("A")), "rows=" + rows,
+			"cols=" + cols, "out_S=" + output("S"), "out_R=" + output("R"), "out_C=" + output("C")};
 
 		runTest(true, (expectedException != null), expectedException, -1);
 
@@ -197,7 +197,8 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 		if ( expectedException == null )
 			compareResults(1e-11);
 
-		assert(checkedPrivacyConstraintsContains(privacyLevel));
+		assertTrue("The privacy level " + privacyLevel.toString() + " should have been checked during execution",
+			checkedPrivacyConstraintsContains(privacyLevel));
 
 		TestUtils.shutdownThread(t);
 		rtplatform = platformOld;
@@ -206,12 +207,12 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 
 	@Test
 	public void transferPrivateTest() {
-		federatedRCBind(Types.ExecMode.SINGLE_NODE, PrivacyLevel.Private, DMLException.class);
+		federatedRCBind(Types.ExecMode.SINGLE_NODE, PrivacyLevel.Private, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void transferPrivateAggregationTest() {
-		federatedRCBind(Types.ExecMode.SINGLE_NODE, PrivacyLevel.PrivateAggregation, DMLException.class);
+		federatedRCBind(Types.ExecMode.SINGLE_NODE, PrivacyLevel.PrivateAggregation, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -248,8 +249,9 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 		TestConfiguration config = availableTestConfigurations.get("transfer");
 		loadTestConfiguration(config);
 		fullDMLScriptName = HOME + TRANSFER_TEST_NAME + ".dml";
-		programArgs = new String[] {"-checkPrivacy", "-args", "\"localhost:" + port + "/" + input("A") + "\"", Integer.toString(rows),
-			Integer.toString(cols), output("R"), output("C")};
+		programArgs = new String[] {"-checkPrivacy", "-nvargs",
+			"in=" + TestUtils.federatedAddress(port, input("A")), "rows=" + rows,
+			"cols=" + cols, "out_R=" + output("R"), "out_C=" + output("C")};
 
 		runTest(true, (expectedException != null), expectedException, -1);
 
@@ -266,12 +268,12 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 
 	@Test
 	public void matVecMultPrivateTest() {
-		federatedMultiply(Types.ExecMode.SINGLE_NODE, PrivacyLevel.Private, DMLException.class);
+		federatedMultiply(Types.ExecMode.SINGLE_NODE, PrivacyLevel.Private, DMLRuntimeException.class);
 	}
 
 	@Test
 	public void matVecMultPrivateAggregationTest() {
-		federatedMultiply(Types.ExecMode.SINGLE_NODE, PrivacyLevel.PrivateAggregation, DMLException.class);
+		federatedMultiply(Types.ExecMode.SINGLE_NODE, PrivacyLevel.PrivateAggregation, DMLRuntimeException.class);
 	}
 
 	@Test
@@ -287,7 +289,7 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 			DMLScript.USE_LOCAL_SPARK_CONFIG = true;
 		}
 
-		Process t1, t2;
+		Thread t1, t2;
 
 		getAndLoadTestConfiguration("matvecmult");
 		String HOME = SCRIPT_DIR + TEST_DIR;
@@ -308,8 +310,8 @@ public class FederatedWorkerHandlerTest extends AutomatedTestBase {
 
 		int port1 = getRandomAvailablePort();
 		int port2 = getRandomAvailablePort();
-		t1 = startLocalFedWorker(port1);
-		t2 = startLocalFedWorker(port2);
+		t1 = startLocalFedWorkerThread(port1);
+		t2 = startLocalFedWorkerThread(port2);
 
 		TestConfiguration config = availableTestConfigurations.get("matvecmult");
 		loadTestConfiguration(config);
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/FineGrainedPrivacyTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/FineGrainedPrivacyTest.java
new file mode 100644
index 0000000..981a2d9
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/FineGrainedPrivacyTest.java
@@ -0,0 +1,191 @@
+/*
+ * 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.privacy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+import org.apache.sysds.runtime.privacy.finegrained.DataRange;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacy;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacyList;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacyMap;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+import org.junit.runner.RunWith;
+
+@RunWith(Parameterized.class)
+public class FineGrainedPrivacyTest {
+
+	private FineGrainedPrivacy constraints;
+
+	public FineGrainedPrivacyTest(FineGrainedPrivacy constraints){
+		this.constraints = constraints;
+	}
+
+	@Parameterized.Parameters
+	public static Collection<FineGrainedPrivacy[]> FineGrainedPrivacy(){
+		return Arrays.asList(new FineGrainedPrivacy[][] {
+			{new FineGrainedPrivacyMap()},
+			{new FineGrainedPrivacyList()}
+		});
+	}
+
+	@After
+	public void setConstraintsToNull(){
+		constraints.removeAllConstraints();
+		constraints = null;
+	}
+
+	@Test
+	public void getPrivacyLevelSingleConstraintCompletelyInRangeTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevel(new DataRange(new long[]{4L,4L,7L}, new long[]{5L,5L,8L}));
+		assertTrue(outputMap.containsKey(inputDataRange));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelSingleConstraintInRangeTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevel(new DataRange(new long[]{1L,4L,7L}, new long[]{5L,5L,8L}));
+		assertTrue(outputMap.containsKey(inputDataRange));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelSingleConstraintNotInRangeTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevel(new DataRange(new long[]{0L,4L,7L}, new long[]{2L,5L,8L}));
+		assertFalse(outputMap.containsKey(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelMultiConstraintInRangeTest(){
+		DataRange inputDataRange1 = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		DataRange inputDataRange2 = new DataRange(new long[]{10L,14L,12L}, new long[]{45L,23L,15L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange1, inputPrivacyLevel);
+		constraints.put(inputDataRange2, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevel(new DataRange(new long[]{4L,3L,8L}, new long[]{50L,55L,19L}));
+		assertTrue(outputMap.containsKey(inputDataRange1));
+		assertTrue(outputMap.containsKey(inputDataRange2));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange1));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange2));
+	}
+
+	@Test
+	public void getPrivacyLevelOfElementSingleConstraintTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevelOfElement(new long[]{4L,4L,7L});
+		assertTrue(outputMap.containsKey(inputDataRange));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelOfElementOnLowerBoundSingleConstraintTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevelOfElement(new long[]{3L,2L,7L});
+		assertTrue(outputMap.containsKey(inputDataRange));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelOfElementOnUpperBoundSingleConstraintTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevelOfElement(new long[]{5L,6L,9L});
+		assertTrue(outputMap.containsKey(inputDataRange));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelOfElementSingleConstraintNotInRangeTest(){
+		DataRange inputDataRange = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevelOfElement(new long[]{7L,10L,1L});
+		assertFalse(outputMap.containsKey(inputDataRange));
+	}
+
+	@Test
+	public void getPrivacyLevelOfElementDoubleConstraintInSingleTest(){
+		DataRange inputDataRange1 = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		DataRange inputDataRange2 = new DataRange(new long[]{10L,14L,12L}, new long[]{45L,23L,15L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange1, inputPrivacyLevel);
+		constraints.put(inputDataRange2, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevelOfElement(new long[]{21L,17L,13L});
+		assertTrue("inputDataRange2 should be in outputMap since element is in the range", outputMap.containsKey(inputDataRange2));
+		assertFalse("inputDataRange1 should not be in outputMap", outputMap.containsKey(inputDataRange1));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange2));
+	}
+
+	@Test
+	public void getPrivacyLevelOfElementDoubleConstraintInBothTest(){
+		DataRange inputDataRange1 = new DataRange(new long[]{3L,2L,7L}, new long[]{5L,6L,9L});
+		DataRange inputDataRange2 = new DataRange(new long[]{1,1L,8L}, new long[]{45L,23L,15L});
+		PrivacyLevel inputPrivacyLevel = PrivacyLevel.Private;
+		constraints.put(inputDataRange1, inputPrivacyLevel);
+		constraints.put(inputDataRange2, inputPrivacyLevel);
+		Map<DataRange, PrivacyLevel> outputMap = constraints.getPrivacyLevelOfElement(new long[]{4L,4L,9L});
+		assertTrue("inputDataRange2 should be in outputMap since element is in the range", outputMap.containsKey(inputDataRange2));
+		assertTrue("inputDataRange1 should be in outputMap since element is in the range", outputMap.containsKey(inputDataRange1));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange2));
+		assertEquals(inputPrivacyLevel, outputMap.get(inputDataRange1));
+	}
+
+	@Test
+	public void getDataRangesOfPrivacyLevelTest(){
+		DataRange inputDataRange1 = new DataRange(new long[]{60L,30L,70L}, new long[]{90L,60L,150L});
+		DataRange inputDataRange2 = new DataRange(new long[]{10,10L,18L}, new long[]{450L,230L,250L});
+		DataRange inputDataRange3 = new DataRange(new long[]{300L,250L,740L}, new long[]{520L,630L,1090L});
+		DataRange inputDataRange4 = new DataRange(new long[]{10,10L,10L}, new long[]{30L,40L,50L});
+		PrivacyLevel inputPrivacyLevel1 = PrivacyLevel.Private;
+		PrivacyLevel inputPrivacyLevel2 = PrivacyLevel.PrivateAggregation;
+		constraints.put(inputDataRange1, inputPrivacyLevel1);
+		constraints.put(inputDataRange2, inputPrivacyLevel1);
+		constraints.put(inputDataRange3, inputPrivacyLevel2);
+		constraints.put(inputDataRange4, inputPrivacyLevel2);
+		DataRange[] outputDataRanges1 = constraints.getDataRangesOfPrivacyLevel(inputPrivacyLevel1);
+		assertEquals(inputDataRange1, outputDataRanges1[0]);
+		assertEquals(inputDataRange2, outputDataRanges1[1]);
+		DataRange[] outputDataRanges2 = constraints.getDataRangesOfPrivacyLevel(inputPrivacyLevel2);
+		assertEquals(inputDataRange3, outputDataRanges2[0]);
+		assertEquals(inputDataRange4, outputDataRanges2[1]);
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/GLMTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/GLMTest.java
index 8039a66..5919523 100644
--- a/src/test/java/org/apache/sysds/test/functions/privacy/GLMTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/GLMTest.java
@@ -55,6 +55,7 @@ public class GLMTest extends AutomatedTestBase
 	protected double distParam, linkPower, intercept, logFeatureVarianceDisbalance, avgLinearForm, stdevLinearForm, dispersion;
 
 	protected GLMType glmType;
+	protected final static boolean runAll = false;
 
 	public enum GLMType {
 		Gaussianlog,
@@ -154,7 +155,10 @@ public class GLMTest extends AutomatedTestBase
 			{  100,  10,  2, -1.0,  4,  0.0,  0.01, 3.0,  -2.0,  1.0,  1.0, GLMType.Bernoullicloglog1 }, // Bernoulli {-1, 1}.cloglog
 			{  200,  10,  2, -1.0,  5,  0.0,  0.01, 3.0,   0.0,  2.0,  1.0, GLMType.Bernoullicauchit },  // Bernoulli {-1, 1}.cauchit
 		};
-		return Arrays.asList(data);
+		if ( runAll )
+			return Arrays.asList(data);
+		else 
+			return Arrays.asList( new Object[][]{data[0]} );
 	}
 
 	@Override
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/MatrixMultiplicationPropagationTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/MatrixMultiplicationPropagationTest.java
index 0715b0a..9e5110d 100644
--- a/src/test/java/org/apache/sysds/test/functions/privacy/MatrixMultiplicationPropagationTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/MatrixMultiplicationPropagationTest.java
@@ -109,8 +109,10 @@ public class MatrixMultiplicationPropagationTest extends AutomatedTestBase {
 		compareResults(1e-9);
 
 		// Check that the output metadata is correct
-		String actualPrivacyValue = readDMLMetaDataValue("c", OUTPUT_DIR, DataExpression.PRIVACY);
-		assertEquals(String.valueOf(privacyLevel), actualPrivacyValue);
+		if ( privacyLevel != PrivacyLevel.None ){
+			String actualPrivacyValue = readDMLMetaDataValue("c", OUTPUT_DIR, DataExpression.PRIVACY);
+			assertEquals(String.valueOf(privacyLevel), actualPrivacyValue);
+		} else exceptionExpected("c", OUTPUT_DIR);
 	}
 
 	@Test
@@ -142,9 +144,18 @@ public class MatrixMultiplicationPropagationTest extends AutomatedTestBase {
 
 		// Check that a JSONException is thrown 
 		// because no privacy metadata should be written to c
+		exceptionExpected("c", OUTPUT_DIR);
+	}
+
+	/**
+	 * Check that an exception is thrown when metadata of variable is read.
+	 * @param variable name of variable
+	 * @param dir directory of variable
+	 */
+	private static void exceptionExpected(String variable, String dir){
 		boolean JSONExceptionThrown = false;
 		try{
-			readDMLMetaDataValue("c", OUTPUT_DIR, DataExpression.PRIVACY);
+			readDMLMetaDataValue(variable, dir, DataExpression.PRIVACY);
 		} catch (JSONException e){
 			JSONExceptionThrown = true;
 		} catch (Exception e){
@@ -180,7 +191,10 @@ public class MatrixMultiplicationPropagationTest extends AutomatedTestBase {
 		
 		writeInputMatrixWithMTD("a", a, false, dataCharacteristics, privacyConstraint);
 
-		String actualPrivacyValue = readDMLMetaDataValue("a", INPUT_DIR, DataExpression.PRIVACY);
-		assertEquals(String.valueOf(privacyLevel), actualPrivacyValue);
+		if ( privacyLevel != PrivacyLevel.None ){
+			String actualPrivacyValue = readDMLMetaDataValue("a", INPUT_DIR, DataExpression.PRIVACY);
+			assertEquals(String.valueOf(privacyLevel), actualPrivacyValue);
+		} else exceptionExpected("a", INPUT_DIR);
+		
 	}
 }
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/PrivacyPropagatorTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/PrivacyPropagatorTest.java
new file mode 100644
index 0000000..1d9f833
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/PrivacyPropagatorTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.privacy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map;
+
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint;
+import org.apache.sysds.runtime.privacy.PrivacyPropagator;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+import org.apache.sysds.runtime.privacy.finegrained.DataRange;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.junit.Test;
+
+public class PrivacyPropagatorTest extends AutomatedTestBase {
+
+	@Override
+	public void setUp() {
+		// TODO Auto-generated method stub
+
+	}
+
+	@Test
+	public void matrixMultiplicationPropagationTestPrivateGeneral(){
+		MatrixBlock inputMatrix1 = new MatrixBlock(10,20,15);
+		MatrixBlock inputMatrix2 = new MatrixBlock(20,30,12);
+		PrivacyConstraint constraint1 = new PrivacyConstraint(PrivacyLevel.Private);
+		constraint1.getFineGrainedPrivacy().put(new DataRange(new long[]{3,8},new long[]{2,5}), PrivacyLevel.Private);
+		PrivacyConstraint constraint2 = new PrivacyConstraint();
+		PrivacyConstraint mergedConstraint = PrivacyPropagator.matrixMultiplicationPropagation(inputMatrix1, constraint1, inputMatrix2, constraint2);
+		assertTrue("Privacy should be set to Private", mergedConstraint.hasPrivateElements());
+		assertFalse("Fine grained constraint should not be propagated", mergedConstraint.hasFineGrainedConstraints());
+	}
+
+	@Test
+	public void matrixMultiplicationPropagationTestPrivateGeneral2(){
+		MatrixBlock inputMatrix1 = new MatrixBlock(10,20,15);
+		MatrixBlock inputMatrix2 = new MatrixBlock(20,30,12);
+		PrivacyConstraint constraint1 = new PrivacyConstraint(PrivacyLevel.Private);
+		constraint1.getFineGrainedPrivacy().put(new DataRange(new long[]{3,8},new long[]{2,5}), PrivacyLevel.PrivateAggregation);
+		PrivacyConstraint constraint2 = new PrivacyConstraint();
+		PrivacyConstraint mergedConstraint = PrivacyPropagator.matrixMultiplicationPropagation(inputMatrix1, constraint1, inputMatrix2, constraint2);
+		assertTrue("Privacy should be set to Private", mergedConstraint.hasPrivateElements());
+		assertFalse("Fine grained constraint should not be propagated", mergedConstraint.hasFineGrainedConstraints());
+	}
+
+	@Test
+	public void matrixMultiplicationPropagationTestPrivateFineGrained(){
+		MatrixBlock inputMatrix1 = new MatrixBlock(4,3,2);
+		MatrixBlock inputMatrix2 = new MatrixBlock(3,3,4);
+		PrivacyConstraint constraint1 = new PrivacyConstraint();
+		constraint1.getFineGrainedPrivacy().put(new DataRange(new long[]{1,0},new long[]{1,1}), PrivacyLevel.Private);
+		PrivacyConstraint constraint2 = new PrivacyConstraint();
+		PrivacyConstraint mergedConstraint = PrivacyPropagator.matrixMultiplicationPropagation(inputMatrix1, constraint1, inputMatrix2, constraint2);
+		assertTrue("Privacy should be set to Private", mergedConstraint.hasPrivateElements());
+		assertTrue("Fine grained constraint should not be propagated", mergedConstraint.hasFineGrainedConstraints());
+		assertTrue("Merged constraint should not contain privacy level PrivateAggregation", mergedConstraint.getFineGrainedPrivacy().getDataRangesOfPrivacyLevel(PrivacyLevel.PrivateAggregation).length == 0);
+		Map<DataRange, PrivacyLevel> outputElement1 = mergedConstraint.getFineGrainedPrivacy().getPrivacyLevelOfElement(new long[]{1,0});
+		Map<DataRange, PrivacyLevel> outputElement2 = mergedConstraint.getFineGrainedPrivacy().getPrivacyLevelOfElement(new long[]{1,1});
+		Map<DataRange, PrivacyLevel> outputElement3 = mergedConstraint.getFineGrainedPrivacy().getPrivacyLevelOfElement(new long[]{1,2});
+		assertEquals(1, outputElement1.size());
+		assertEquals(1, outputElement2.size());
+		assertEquals(1, outputElement3.size());
+		assertTrue("Privacy level of element 1 is Private", outputElement1.containsValue(PrivacyLevel.Private));
+		assertTrue("Privacy level of element 2 is Private", outputElement2.containsValue(PrivacyLevel.Private));
+		assertTrue("Privacy level of element 3 is Private", outputElement3.containsValue(PrivacyLevel.Private));
+		assertTrue("Any other index has no privacy constraint", mergedConstraint.getFineGrainedPrivacy().getPrivacyLevel(new DataRange(new long[]{2,0}, new long[]{3,2})).isEmpty() );
+	}
+
+	@Test
+	public void matrixMultiplicationPropagationTestPrivateFineGrained2(){
+		MatrixBlock inputMatrix1 = new MatrixBlock(4,3,2);
+		MatrixBlock inputMatrix2 = new MatrixBlock(3,3,4);
+		PrivacyConstraint constraint1 = new PrivacyConstraint();
+		PrivacyConstraint constraint2 = new PrivacyConstraint();
+		constraint2.getFineGrainedPrivacy().put(new DataRange(new long[]{1,0},new long[]{1,1}), PrivacyLevel.Private);
+		PrivacyConstraint mergedConstraint = PrivacyPropagator.matrixMultiplicationPropagation(inputMatrix1, constraint1, inputMatrix2, constraint2);
+		assertTrue("Privacy should be set to Private", mergedConstraint.hasPrivateElements());
+		assertTrue("Fine grained constraint should not be propagated", mergedConstraint.hasFineGrainedConstraints());
+		assertTrue("Merged constraint should not contain privacy level PrivateAggregation", mergedConstraint.getFineGrainedPrivacy().getDataRangesOfPrivacyLevel(PrivacyLevel.PrivateAggregation).length == 0);
+		Map<DataRange, PrivacyLevel> outputRange = mergedConstraint.getFineGrainedPrivacy().getPrivacyLevel(new DataRange(new long[]{0,0},new long[]{3,1}));
+		assertEquals(8, outputRange.size());
+		assertTrue("Privacy level is Private", outputRange.containsValue(PrivacyLevel.Private));
+		assertTrue("Any other index has no privacy constraint", mergedConstraint.getFineGrainedPrivacy().getPrivacyLevel(new DataRange(new long[]{0,2}, new long[]{3,2})).isEmpty() );
+	}
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/privacy/ReadWriteTest.java b/src/test/java/org/apache/sysds/test/functions/privacy/ReadWriteTest.java
new file mode 100644
index 0000000..02bcc31
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/privacy/ReadWriteTest.java
@@ -0,0 +1,178 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.functions.privacy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import org.apache.sysds.runtime.meta.MatrixCharacteristics;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint;
+import org.apache.sysds.runtime.privacy.PrivacyConstraint.PrivacyLevel;
+import org.apache.sysds.runtime.privacy.finegrained.DataRange;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacy;
+import org.apache.sysds.runtime.privacy.finegrained.FineGrainedPrivacyList;
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+import org.apache.wink.json4j.JSONObject;
+import org.junit.Test;
+
+public class ReadWriteTest extends AutomatedTestBase {
+
+	private static final String TEST_DIR = "functions/privacy/";
+	private final static String TEST_CLASS_DIR = TEST_DIR + ReadWriteTest.class.getSimpleName() + "/";
+
+	private final int n = 10;
+	private final int m = 20;
+
+	@Override
+	public void setUp() {
+		addTestConfiguration("ReadWriteTest",
+			new TestConfiguration(TEST_CLASS_DIR, "ReadWriteTest", new String[]{}));
+		addTestConfiguration("ReadWriteTest2",
+			new TestConfiguration(TEST_CLASS_DIR, "ReadWriteTest2", new String[]{"b"}));
+		addTestConfiguration("serialize",
+			new TestConfiguration(TEST_CLASS_DIR, "serialize", new String[]{}));
+	}
+
+
+	@Test
+	public void writeFineGrainedPrivacyMetadataTest(){
+		TestConfiguration config = availableTestConfigurations.get("ReadWriteTest");
+		loadTestConfiguration(config);
+
+		writeA();
+		
+		JSONObject metadata = getMetaDataJSON("a", "in/");
+		assertTrue(metadata.containsKey("fine_grained_privacy"));
+	}
+
+	@Test
+	public void readAndWriteFineGrainedConstraintsTest(){
+		TestConfiguration config = availableTestConfigurations.get("ReadWriteTest2");
+		loadTestConfiguration(config);
+		fullDMLScriptName = SCRIPT_DIR + TEST_DIR + config.getTestScript() + ".dml";
+
+		double[][] a = writeA();
+
+		writeExpectedMatrix("b", a);
+		programArgs = new String[]{"-nvargs",
+		"a=" + input("a"), "b=" + output("b"),
+		"m=" + m, "n=" + n };
+		runTest(true,false,null,-1);
+		compareResults(1e-9);
+
+		JSONObject metadata = getMetaDataJSON("b");
+		assertTrue(metadata.containsKey("fine_grained_privacy"));
+	}
+
+	private double[][] writeA(){
+		int k = 15;
+		double[][] a = getRandomMatrix(m, n, -1, 1, 1, -1);
+
+		PrivacyConstraint privacyConstraint = new PrivacyConstraint();
+		FineGrainedPrivacy fgp = new FineGrainedPrivacyList();
+		fgp.put(new DataRange(new long[]{1,2}, new long[]{5,4}), PrivacyLevel.Private);
+		fgp.put(new DataRange(new long[]{7,1}, new long[]{9,1}), PrivacyLevel.Private);
+		fgp.put(new DataRange(new long[]{10,5}, new long[]{10,9}), PrivacyLevel.PrivateAggregation);
+		privacyConstraint.setFineGrainedPrivacyConstraints(fgp);
+		MatrixCharacteristics dataCharacteristics = new MatrixCharacteristics(m,n,k,k);
+		writeInputMatrixWithMTD("a", a, false, dataCharacteristics, privacyConstraint);
+		return a;
+	}
+
+	@Test
+	public void serializeTest() throws IOException {
+		serializeBaseTest(PrivacyLevel.Private, null);
+	}
+
+	@Test
+	public void serializeFineGrainedTest() throws IOException {
+		FineGrainedPrivacy fineGrained = new FineGrainedPrivacyList();
+		fineGrained.put(new DataRange(new long[]{1,2,3}, new long[]{4,5,6}), PrivacyLevel.Private);
+		serializeBaseTest(PrivacyLevel.PrivateAggregation, fineGrained);
+	}
+
+	@Test
+	public void serializeFineGrainedTest2() throws IOException {
+		FineGrainedPrivacy fineGrained = new FineGrainedPrivacyList();
+		fineGrained.put(new DataRange(new long[]{1,2,3}, new long[]{4,5,6}), PrivacyLevel.Private);
+		fineGrained.put(new DataRange(new long[]{7,8,9}, new long[]{10,11,12}), PrivacyLevel.Private);
+		serializeBaseTest(PrivacyLevel.PrivateAggregation, fineGrained);
+	}
+
+	@Test
+	public void serializeFineGrainedTest3() throws IOException {
+		FineGrainedPrivacy fineGrained = new FineGrainedPrivacyList();
+		fineGrained.put(new DataRange(new long[]{1,2,3}, new long[]{4,5,6}), PrivacyLevel.Private);
+		fineGrained.put(new DataRange(new long[]{7,8,9}, new long[]{10,11,12}), PrivacyLevel.Private);
+		serializeBaseTest(PrivacyLevel.None, fineGrained);
+	}
+
+	@Test
+	public void serializeFineGrainedTest4() throws IOException {
+		FineGrainedPrivacy fineGrained = new FineGrainedPrivacyList();
+		fineGrained.put(new DataRange(new long[]{1,2,3}, new long[]{4,5,6}), PrivacyLevel.PrivateAggregation);
+		fineGrained.put(new DataRange(new long[]{7,8,9}, new long[]{10,11,12}), PrivacyLevel.PrivateAggregation);
+		serializeBaseTest(PrivacyLevel.PrivateAggregation, fineGrained);
+	}
+
+	@Test
+	public void serializeFineGrainedTest5() throws IOException {
+		FineGrainedPrivacy fineGrained = new FineGrainedPrivacyList();
+		fineGrained.put(new DataRange(new long[]{1,2,3}, new long[]{4,5,6}), PrivacyLevel.PrivateAggregation);
+		fineGrained.put(new DataRange(new long[]{7,8,9}, new long[]{10,11,12}), PrivacyLevel.Private);
+		serializeBaseTest(PrivacyLevel.None, fineGrained);
+	}
+
+	private void serializeBaseTest(PrivacyLevel privacyLevel, FineGrainedPrivacy fineGrainedPrivacy) throws IOException {
+		TestConfiguration config = availableTestConfigurations.get("serialize");
+		loadTestConfiguration(config);
+
+		//Writing
+		PrivacyConstraint privacyConstraint = new PrivacyConstraint(privacyLevel);
+		if ( fineGrainedPrivacy != null )
+			privacyConstraint.setFineGrainedPrivacyConstraints(fineGrainedPrivacy);
+		String outputPath = baseDirectory + INPUT_DIR + "serialize.ser";
+		try (FileOutputStream fileOutput = new FileOutputStream(outputPath)){
+			try (ObjectOutputStream outputStream = new ObjectOutputStream(fileOutput)){
+				privacyConstraint.writeExternal(outputStream);
+			}
+		}
+		
+		//Reading
+		PrivacyConstraint loadedConstraint = new PrivacyConstraint();
+		try (FileInputStream fileInput = new FileInputStream(outputPath)){
+			try (ObjectInputStream inputStream = new ObjectInputStream(fileInput)){
+				loadedConstraint.readExternal(inputStream);
+			}
+		}
+		
+		//Comparing
+		assertEquals(privacyConstraint.getPrivacyLevel(), loadedConstraint.getPrivacyLevel());
+		assertEquals(privacyConstraint.getFineGrainedPrivacy(), loadedConstraint.getFineGrainedPrivacy());
+	}
+	
+}
\ No newline at end of file
diff --git a/src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml b/src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml
index 9705cef..0d4952a 100644
--- a/src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml
+++ b/src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml
@@ -19,8 +19,6 @@
 #
 #-------------------------------------------------------------
 
-# junit test class: org.tugraz.sysds.test.integration.functions.binary.matrix.MatrixMultiplicationTest.java
-
 A = read($a, rows=$m, cols=$n, format="text");
 B = read($b, rows=$n, cols=$k, format="text");
 C = A %*% B;
diff --git a/src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml b/src/test/scripts/functions/privacy/ReadWriteTest2.dml
similarity index 83%
copy from src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml
copy to src/test/scripts/functions/privacy/ReadWriteTest2.dml
index 9705cef..b4b3c11 100644
--- a/src/test/scripts/functions/privacy/MatrixMultiplicationPropagationTest.dml
+++ b/src/test/scripts/functions/privacy/ReadWriteTest2.dml
@@ -19,9 +19,6 @@
 #
 #-------------------------------------------------------------
 
-# junit test class: org.tugraz.sysds.test.integration.functions.binary.matrix.MatrixMultiplicationTest.java
-
 A = read($a, rows=$m, cols=$n, format="text");
-B = read($b, rows=$n, cols=$k, format="text");
-C = A %*% B;
-write(C, $c, format="text");
\ No newline at end of file
+B = A;
+write(B, $b, format="text");