You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@systemds.apache.org by ar...@apache.org on 2021/05/06 21:20:18 UTC
[systemds] branch master updated: [SYSTEMDS-2973] Distributed read,
write support for LIBSVM files
This is an automated email from the ASF dual-hosted git repository.
arnabp20 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 a371a65 [SYSTEMDS-2973] Distributed read, write support for LIBSVM files
a371a65 is described below
commit a371a654eeb07a206c743a6a1a83e5f86e8ae1c5
Author: Saeed <s....@gmail.com>
AuthorDate: Thu May 6 23:15:48 2021 +0200
[SYSTEMDS-2973] Distributed read, write support for LIBSVM files
This patch adds read, write of LIBSVM files on Spark.
- Maps RDD partitions of LIBSVM rows into a set of partial binary blocks
- Supports custom delimiters between data instances and index value
Currently only matrix read is supported. frame support is left as future work.
Closes #1232, closes #1257.
---
src/main/java/org/apache/sysds/lops/Data.java | 176 ++-
.../org/apache/sysds/parser/DataExpression.java | 100 +-
.../runtime/instructions/SPInstructionParser.java | 143 +-
.../cp/ParameterizedBuiltinCPInstruction.java | 6 +-
.../instructions/cp/VariableCPInstruction.java | 330 ++--
.../spark/LIBSVMReblockSPInstruction.java | 135 ++
.../instructions/spark/ReblockSPInstruction.java | 66 +-
.../runtime/instructions/spark/SPInstruction.java | 30 +-
.../instructions/spark/WriteSPInstruction.java | 112 +-
.../spark/utils/RDDConverterUtils.java | 662 +++++---
.../runtime/io/FileFormatPropertiesLIBSVM.java | 89 ++
.../sysds/runtime/io/MatrixReaderFactory.java | 37 +-
.../sysds/runtime/io/MatrixWriterFactory.java | 26 +-
.../apache/sysds/runtime/io/ReaderTextLIBSVM.java | 88 +-
.../sysds/runtime/io/ReaderTextLIBSVMParallel.java | 108 +-
.../apache/sysds/runtime/io/WriterTextLIBSVM.java | 55 +-
.../sysds/runtime/io/WriterTextLIBSVMParallel.java | 30 +-
.../test/functions/data/misc/NoRenameTest.java | 20 +-
.../test/functions/io/libsvm/LIBSVMConfig.java | 77 +
.../test/functions/io/libsvm/ReadLIBSVMTest.java | 88 +-
.../test/functions/io/libsvm/ReadLIBSVMTest1.java | 42 +
.../test/functions/io/libsvm/ReadLIBSVMTest2.java | 42 +
.../test/functions/io/libsvm/ReadLIBSVMTest3.java | 42 +
.../functions/io/libsvm/ReadLIBSVMTestBase.java | 38 +
.../test/functions/io/libsvm/WriteLIBSVMTest.java | 130 +-
.../test/functions/io/libsvm/WriteLIBSVMTest1.java | 42 +
.../test/functions/io/libsvm/WriteLIBSVMTest2.java | 43 +
.../functions/io/libsvm/WriteLIBSVMTestBase.java | 40 +
.../functions/io/libsvm/ReadLIBSVMTest_1.dml | 2 +-
.../functions/io/libsvm/ReadLIBSVMTest_2.dml | 3 +-
.../{ReadLIBSVMTest_1.dml => ReadLIBSVMTest_3.dml} | 4 +-
.../functions/io/libsvm/WriteLIBSVMTest_1.dml | 10 +-
.../functions/io/libsvm/WriteLIBSVMTest_2.dml | 7 +-
.../functions/io/libsvm/in/transfusion_1.libsvm | 6 +
.../io/libsvm/in/transfusion_1.libsvm.mtd | 11 +
.../functions/io/libsvm/in/transfusion_2.libsvm | 1605 ++++++++++++++++++++
.../functions/io/libsvm/in/transfusion_3.libsvm | 6 +
.../functions/io/libsvm/in/transfusion_W1.libsvm | 6 +
.../io/libsvm/in/transfusion_W1.libsvm.mtd | 11 +
.../functions/io/libsvm/in/transfusion_W2.libsvm | 1605 ++++++++++++++++++++
.../io/libsvm/in/transfusion_W2.libsvm.mtd | 11 +
.../{WriteLIBSVMTest_1.dml => libsvm_verify.R} | 44 +-
42 files changed, 5276 insertions(+), 852 deletions(-)
diff --git a/src/main/java/org/apache/sysds/lops/Data.java b/src/main/java/org/apache/sysds/lops/Data.java
index b2b6e0c..928d595 100644
--- a/src/main/java/org/apache/sysds/lops/Data.java
+++ b/src/main/java/org/apache/sysds/lops/Data.java
@@ -51,7 +51,7 @@ public class Data extends Lop
// All literals have default format type of TEXT
return new Data(OpOpData.PERSISTENTREAD, null, null, null, literalValue, DataType.SCALAR, vt, FileFormat.BINARY);
}
-
+
/**
* Constructor to setup read or write LOP
* In case of write: <code>input</code> must be provided. This will always be added as the first element in <code>input</code> array.
@@ -66,8 +66,8 @@ public class Data extends Lop
* @param vt value type
* @param fmt file format
*/
- public Data(OpOpData op, Lop input, HashMap<String, Lop>
- inputParametersLops, String name, String literal, DataType dt, ValueType vt, FileFormat fmt)
+ public Data(OpOpData op, Lop input, HashMap<String, Lop>
+ inputParametersLops, String name, String literal, DataType dt, ValueType vt, FileFormat fmt)
{
super(Lop.Type.Data, dt, vt);
_op = op;
@@ -99,15 +99,15 @@ public class Data extends Lop
addInput(input);
input.addOutput(this);
}
-
+
_inputParams = inputParametersLops;
-
+
if ( _inputParams != null ) {
for (Lop lop : inputParametersLops.values()) {
addInput(lop);
lop.addOutput(this);
}
- if ( inputParametersLops.get(DataExpression.IO_FILENAME)!= null
+ if ( inputParametersLops.get(DataExpression.IO_FILENAME)!= null
&& inputParametersLops.get(DataExpression.IO_FILENAME) instanceof Data )
{
OutputParameters outParams = ((Data)inputParametersLops.get(DataExpression.IO_FILENAME)).getOutputParameters();
@@ -115,7 +115,7 @@ public class Data extends Lop
this.getOutputParameters().setFile_name(fName);
}
}
-
+
//set output format
formatType = fmt;
outParams.setFormat(fmt);
@@ -125,11 +125,11 @@ public class Data extends Lop
private void setLopProperties() {
lps.setProperties ( inputs, ExecType.INVALID);
}
-
+
/**
* Data-Lop-specific method to set the execution type for persistent write.
- * TODO: split lops into MR/CP lop.
- *
+ * TODO: split lops into MR/CP lop.
+ *
* @param et execution type
*/
public void setExecType( ExecType et ) {
@@ -137,13 +137,13 @@ public class Data extends Lop
}
/**
- * method to get format type for input, output files.
+ * method to get format type for input, output files.
* @return file format
*/
public FileFormat getFileFormatType() {
return formatType ;
}
-
+
@Override
public String toString() {
return getID() + ":" + "File_Name: " + getOutputParameters().getFile_name() + " "
@@ -157,30 +157,30 @@ public class Data extends Lop
* method to get operation type, i.e. read/write.
* @return operation type
*/
-
+
public OpOpData getOperationType() {
return _op;
}
-
+
/**
- * method to get inputParams
+ * method to get inputParams
* @return input parameters
*/
public HashMap<String, Lop> getInputParams(){
return _inputParams;
}
-
+
public Lop getNamedInputLop(String name) {
return _inputParams.get(name);
}
-
+
public Lop getNamedInputLop(String name, String defaultVal) {
if( _inputParams.containsKey(name) )
return _inputParams.get(name);
else
return Data.createLiteralLop(ValueType.STRING, defaultVal);
}
-
+
/**
* method to check if this data lop represents a literal.
* @return true if data lop is a literal
@@ -188,7 +188,7 @@ public class Data extends Lop
public boolean isLiteral() {
return literal_var;
}
-
+
public boolean getBooleanValue() {
if(literal_var) {
return Boolean.parseBoolean(getOutputParameters().getLabel());
@@ -196,7 +196,7 @@ public class Data extends Lop
else
throw new LopsException("Cannot obtain the value of a non-literal variable at compile time.");
}
-
+
public double getDoubleValue() {
if(literal_var) {
return Double.parseDouble(getOutputParameters().getLabel());
@@ -204,7 +204,7 @@ public class Data extends Lop
else
throw new LopsException("Cannot obtain the value of a non-literal variable at compile time.");
}
-
+
public long getLongValue() {
if(literal_var) {
ValueType vt = getValueType();
@@ -213,7 +213,7 @@ public class Data extends Lop
return Long.parseLong(getOutputParameters().getLabel());
case FP64:
return (long) Double.parseDouble(getOutputParameters().getLabel());
-
+
default:
throw new LopsException("Encountered a non-numeric value " + (vt) + ", while a numeric value is expected.");
}
@@ -221,7 +221,7 @@ public class Data extends Lop
else
throw new LopsException("Can not obtain the value of a non-literal variable at compile time.");
}
-
+
public String getStringValue() {
if(literal_var) {
return getOutputParameters().getLabel();
@@ -229,28 +229,28 @@ public class Data extends Lop
else
throw new LopsException("Cannot obtain the value of a non-literal variable at compile time.");
}
-
+
public boolean isPersistentWrite() {
return _op == OpOpData.PERSISTENTWRITE;
}
-
+
public boolean isPersistentRead() {
return _op == OpOpData.PERSISTENTREAD
&& !literal_var;
}
-
+
/**
* Method to get CP instructions for reading/writing scalars and matrices from/to HDFS.
* This method generates CP read/write instructions.
*/
@Override
- public String getInstructions(String input1, String input2)
+ public String getInstructions(String input1, String input2)
{
- if ( getOutputParameters().getFile_name() == null && _op.isRead() )
+ if ( getOutputParameters().getFile_name() == null && _op.isRead() )
throw new LopsException(this.printErrorLocation() + "Data.getInstructions(): Exepecting a SCALAR data type, encountered " + getDataType());
-
+
StringBuilder sb = new StringBuilder();
- if(this.getExecType() == ExecType.SPARK)
+ if(this.getExecType() == ExecType.SPARK)
sb.append( "SPARK" );
else
sb.append( "CP" );
@@ -267,12 +267,12 @@ public class Data extends Lop
}
else
throw new LopsException(this.printErrorLocation() + "In Data Lop, Unknown operation: " + _op);
-
+
sb.append( OPERAND_DELIMITOR );
Lop fnameLop = _inputParams.get(DataExpression.IO_FILENAME);
boolean literal = (fnameLop instanceof Data && ((Data)fnameLop).isLiteral());
- sb.append ( prepOperand(input2, DataType.SCALAR, ValueType.STRING, literal) );
-
+ sb.append ( prepOperand(input2, DataType.SCALAR, ValueType.STRING, literal) );
+
// attach outputInfo in case of matrices
OutputParameters oparams = getOutputParameters();
if ( _op.isWrite() ) {
@@ -282,14 +282,14 @@ public class Data extends Lop
fmt = oparams.getFormat().toString();
else // scalars will always be written in text format
fmt = FileFormat.TEXT.toString();
-
+
sb.append( prepOperand(fmt, DataType.SCALAR, ValueType.STRING, true));
-
+
if(oparams.getFormat() == FileFormat.CSV) {
Data headerLop = (Data) getNamedInputLop(DataExpression.DELIM_HAS_HEADER_ROW);
Data delimLop = (Data) getNamedInputLop(DataExpression.DELIM_DELIMITER);
Data sparseLop = (Data) getNamedInputLop(DataExpression.DELIM_SPARSE);
-
+
if (headerLop.isVariable())
throw new LopsException(this.printErrorLocation()
+ "Parameter " + DataExpression.DELIM_HAS_HEADER_ROW
@@ -302,33 +302,52 @@ public class Data extends Lop
throw new LopsException(this.printErrorLocation()
+ "Parameter " + DataExpression.DELIM_SPARSE
+ " must be a literal for a seq operation.");
-
+
sb.append(OPERAND_DELIMITOR);
sb.append(headerLop.getBooleanValue());
sb.append(OPERAND_DELIMITOR);
sb.append(delimLop.getStringValue());
sb.append(OPERAND_DELIMITOR);
sb.append(sparseLop.getBooleanValue());
-
- if ( this.getExecType() == ExecType.SPARK )
+
+ if ( this.getExecType() == ExecType.SPARK )
{
sb.append(OPERAND_DELIMITOR);
sb.append(true); //isInputMatrixBlock
}
}
-
+
if(oparams.getFormat() == FileFormat.LIBSVM) {
+ Data delimLop = (Data) getNamedInputLop(DataExpression.DELIM_DELIMITER);
+ Data indexDelimLop = (Data) getNamedInputLop(DataExpression.LIBSVM_INDEX_DELIM);
Data sparseLop = (Data) getNamedInputLop(DataExpression.DELIM_SPARSE);
-
+
+ if(delimLop.isVariable())
+ throw new LopsException(
+ this.printErrorLocation() + "Parameter " + DataExpression.DELIM_DELIMITER + " must be a literal for a seq operation.");
+
+ if(indexDelimLop.isVariable())
+ throw new LopsException(
+ this.printErrorLocation() + "Parameter " + DataExpression.LIBSVM_INDEX_DELIM + " must be a literal for a seq operation.");
+
if (sparseLop.isVariable())
throw new LopsException(this.printErrorLocation()
+ "Parameter " + DataExpression.DELIM_SPARSE
+ " must be a literal for a seq operation.");
sb.append(OPERAND_DELIMITOR);
+ sb.append(delimLop.getStringValue());
+ sb.append(OPERAND_DELIMITOR);
+ sb.append(indexDelimLop.getStringValue());
+ sb.append(OPERAND_DELIMITOR);
sb.append(sparseLop.getBooleanValue());
+
+ if(this.getExecType() == ExecType.SPARK) {
+ sb.append(OPERAND_DELIMITOR);
+ sb.append(true); //isInputMatrixBlock
+ }
}
-
+
}
if (_op.isWrite()) {
@@ -345,36 +364,36 @@ public class Data extends Lop
return sb.toString();
}
-
+
/**
* Method to generate createvar instruction that updates symbol table with metadata, hdfsfile name, etc.
- *
+ *
*/
@Override
public String getInstructions() {
return getCreateVarInstructions(getOutputParameters().getFile_name(), getOutputParameters().getLabel());
}
-
+
@Override
public String getInstructions(String outputFileName) {
return getCreateVarInstructions(outputFileName, getOutputParameters().getLabel() );
}
-
+
public String getCreateVarInstructions(String outputFileName, String outputLabel) {
if ( getDataType() == DataType.MATRIX || getDataType() == DataType.FRAME ) {
-
+
if ( _op.isTransient() )
throw new LopsException("getInstructions() should not be called for transient nodes.");
-
+
OutputParameters oparams = getOutputParameters();
-
+
StringBuilder sb = new StringBuilder();
sb.append( "CP" );
sb.append( OPERAND_DELIMITOR );
sb.append( "createvar" );
- sb.append( OPERAND_DELIMITOR );
+ sb.append( OPERAND_DELIMITOR );
sb.append( outputLabel );
- sb.append( OPERAND_DELIMITOR );
+ sb.append( OPERAND_DELIMITOR );
sb.append( outputFileName );
sb.append( OPERAND_DELIMITOR );
sb.append( false );
@@ -392,17 +411,18 @@ public class Data extends Lop
sb.append( oparams.getNnz() );
sb.append( OPERAND_DELIMITOR );
sb.append( oparams.getUpdateType().toString().toLowerCase() );
-
+
// Format-specific properties
if ( oparams.getFormat() == FileFormat.CSV ) {
sb.append( OPERAND_DELIMITOR );
sb.append( createVarCSVHelper() );
}
// Format-specific properties
- if ( oparams.getFormat() == FileFormat.LIBSVM ) {
+ if ( oparams.getFormat() == FileFormat.LIBSVM ) {
+ sb.append(OPERAND_DELIMITOR);
sb.append( createVarLIBSVMHelper() );
}
-
+
// Frame-specific properties
if( getDataType()==DataType.FRAME ) {
Data schema = (Data) getNamedInputLop(DataExpression.SCHEMAPARAM);
@@ -415,11 +435,11 @@ public class Data extends Lop
throw new LopsException(this.printErrorLocation() + "In Data Lop, Unexpected data type " + getDataType());
}
}
-
+
/**
* Helper function that attaches CSV format-specific properties to createvar instruction.
* The set of properties that are attached for a READ operation is different from that for a WRITE operation.
- *
+ *
* @return instruction with csv format properties appended
*/
private String createVarCSVHelper() {
@@ -427,10 +447,10 @@ public class Data extends Lop
if ( _op.isRead() ) {
Data headerLop = (Data) getNamedInputLop(DataExpression.DELIM_HAS_HEADER_ROW);
Data delimLop = (Data) getNamedInputLop(DataExpression.DELIM_DELIMITER);
- Data fillLop = (Data) getNamedInputLop(DataExpression.DELIM_FILL);
+ Data fillLop = (Data) getNamedInputLop(DataExpression.DELIM_FILL);
Data fillValueLop = (Data) getNamedInputLop(DataExpression.DELIM_FILL_VALUE);
Lop naLop = getNamedInputLop(DataExpression.DELIM_NA_STRINGS);
-
+
sb.append(headerLop.getBooleanValue());
sb.append(OPERAND_DELIMITOR);
sb.append(delimLop.getStringValue());
@@ -452,11 +472,11 @@ public class Data extends Lop
}
}
}
- else { // (operation == OperationTypes.WRITE)
+ else { // (operation == OperationTypes.WRITE)
Data headerLop = (Data) getNamedInputLop(DataExpression.DELIM_HAS_HEADER_ROW);
Data delimLop = (Data) getNamedInputLop(DataExpression.DELIM_DELIMITER);
- Data sparseLop = (Data) getNamedInputLop(DataExpression.DELIM_SPARSE);
-
+ Data sparseLop = (Data) getNamedInputLop(DataExpression.DELIM_SPARSE);
+
if (headerLop.isVariable())
throw new LopsException(this.printErrorLocation()
+ "Parameter " + DataExpression.DELIM_HAS_HEADER_ROW
@@ -469,7 +489,7 @@ public class Data extends Lop
throw new LopsException(this.printErrorLocation()
+ "Parameter " + DataExpression.DELIM_SPARSE
+ " must be a literal for a seq operation.");
-
+
sb.append(headerLop.getBooleanValue());
sb.append(OPERAND_DELIMITOR);
sb.append(delimLop.getStringValue());
@@ -482,21 +502,35 @@ public class Data extends Lop
private String createVarLIBSVMHelper() {
StringBuilder sb = new StringBuilder();
if ( _op.isRead() ) {
- Data naLop = (Data) getNamedInputLop(DataExpression.DELIM_NA_STRINGS);
-
- if ( naLop != null ) {
+ Data delimLop = (Data) getNamedInputLop(DataExpression.DELIM_DELIMITER);
+ Data indexDelimlLop = (Data) getNamedInputLop(DataExpression.LIBSVM_INDEX_DELIM);
+
+ sb.append(delimLop.getStringValue());
sb.append(OPERAND_DELIMITOR);
- sb.append(naLop.getStringValue());
- }
+ sb.append(indexDelimlLop.getStringValue());
+ sb.append(OPERAND_DELIMITOR);
}
- else { // (operation == OperationTypes.WRITE)
- Data sparseLop = (Data) getNamedInputLop(DataExpression.DELIM_SPARSE);
-
+ else { // (operation == OperationTypes.WRITE)
+ Data delimLop = (Data) getNamedInputLop(DataExpression.DELIM_DELIMITER);
+ Data indexDelimLop = (Data) getNamedInputLop(DataExpression.LIBSVM_INDEX_DELIM);
+ Data sparseLop = (Data) getNamedInputLop(DataExpression.DELIM_SPARSE);
+
+ if(delimLop.isVariable())
+ throw new LopsException(
+ this.printErrorLocation() + "Parameter " + DataExpression.DELIM_DELIMITER + " must be a literal for a seq operation.");
+
+ if(indexDelimLop.isVariable())
+ throw new LopsException(
+ this.printErrorLocation() + "Parameter " + DataExpression.LIBSVM_INDEX_DELIM + " must be a literal for a seq operation.");
+
if (sparseLop.isVariable())
throw new LopsException(this.printErrorLocation()
+ "Parameter " + DataExpression.DELIM_SPARSE
+ " must be a literal for a seq operation.");
-
+
+ sb.append(delimLop.getStringValue());
+ sb.append(OPERAND_DELIMITOR);
+ sb.append(indexDelimLop.getStringValue());
sb.append(OPERAND_DELIMITOR);
sb.append(sparseLop.getBooleanValue());
}
diff --git a/src/main/java/org/apache/sysds/parser/DataExpression.java b/src/main/java/org/apache/sysds/parser/DataExpression.java
index 2db8869..408a3fe 100644
--- a/src/main/java/org/apache/sysds/parser/DataExpression.java
+++ b/src/main/java/org/apache/sysds/parser/DataExpression.java
@@ -110,7 +110,8 @@ public class DataExpression extends DataIdentifier
//public static final String DELIM_RECODE = "recode";
public static final String DELIM_NA_STRINGS = "naStrings";
public static final String DELIM_NA_STRING_SEP = "\u00b7";
-
+ // Parameter names relevant to reading/writing delimited index/libsvmv files
+ public static final String LIBSVM_INDEX_DELIM = "indSep";
public static final String DELIM_SPARSE = "sparse"; // applicable only for write
@@ -137,6 +138,8 @@ public class DataExpression extends DataIdentifier
VALUETYPEPARAM, SCHEMAPARAM, DESCRIPTIONPARAM, AUTHORPARAM, CREATEDPARAM,
// Parameters related to delimited/csv files.
DELIM_FILL_VALUE, DELIM_DELIMITER, DELIM_FILL, DELIM_HAS_HEADER_ROW, DELIM_NA_STRINGS,
+ // Parameters related to delimited/libsvm files.
+ LIBSVM_INDEX_DELIM,
// Parameters related to privacy
PRIVACY, FINE_GRAINED_PRIVACY));
@@ -145,7 +148,9 @@ public class DataExpression extends DataIdentifier
Arrays.asList(IO_FILENAME, READROWPARAM, READCOLPARAM, FORMAT_TYPE, DATATYPEPARAM,
VALUETYPEPARAM, SCHEMAPARAM, ROWBLOCKCOUNTPARAM, COLUMNBLOCKCOUNTPARAM, READNNZPARAM,
// Parameters related to delimited/csv files.
- DELIM_FILL_VALUE, DELIM_DELIMITER, DELIM_FILL, DELIM_HAS_HEADER_ROW, DELIM_NA_STRINGS));
+ DELIM_FILL_VALUE, DELIM_DELIMITER, DELIM_FILL, DELIM_HAS_HEADER_ROW, DELIM_NA_STRINGS,
+ // Parameters related to delimited/libsvm files.
+ LIBSVM_INDEX_DELIM));
/* Default Values for delimited (CSV/LIBSVM) files */
public static final String DEFAULT_DELIM_DELIMITER = ",";
@@ -155,6 +160,7 @@ public class DataExpression extends DataIdentifier
public static final boolean DEFAULT_DELIM_SPARSE = false;
public static final String DEFAULT_NA_STRINGS = "";
public static final String DEFAULT_SCHEMAPARAM = "NULL";
+ public static final String DEFAULT_LIBSVM_INDEX_DELIM = ":";
private DataOp _opcode;
private HashMap<String, Expression> _varParams;
@@ -921,6 +927,7 @@ public class DataExpression extends DataIdentifier
|| getVarParam(COLUMNBLOCKCOUNTPARAM) != null
|| getVarParam(FORMAT_TYPE) != null
|| getVarParam(DELIM_DELIMITER) != null
+ || getVarParam(LIBSVM_INDEX_DELIM) != null
|| getVarParam(DELIM_HAS_HEADER_ROW) != null
|| getVarParam(DELIM_FILL) != null
|| getVarParam(DELIM_FILL_VALUE) != null
@@ -1151,33 +1158,55 @@ public class DataExpression extends DataIdentifier
}
}
- boolean islibsvm = false;
- islibsvm = (formatTypeString != null && formatTypeString.equalsIgnoreCase(FileFormat.LIBSVM.toString()));
- if (islibsvm){
+ boolean isLIBSVM = false;
+ isLIBSVM = (formatTypeString != null && formatTypeString.equalsIgnoreCase(FileFormat.LIBSVM.toString()));
+ if (isLIBSVM) {
// Handle libsvm file format
shouldReadMTD = true;
// only allow IO_FILENAME, READROWPARAM, READCOLPARAM
// as valid parameters
- if( !inferredFormatType ){
- for (String key : _varParams.keySet()){
- if (! (key.equals(IO_FILENAME) || key.equals(FORMAT_TYPE)
+ if( !inferredFormatType ) {
+ for (String key : _varParams.keySet()) {
+ if (!(key.equals(IO_FILENAME) || key.equals(FORMAT_TYPE)
|| key.equals(READROWPARAM) || key.equals(READCOLPARAM)
|| key.equals(READNNZPARAM) || key.equals(DATATYPEPARAM)
- || key.equals(VALUETYPEPARAM) ))
+ || key.equals(VALUETYPEPARAM) || key.equals(DELIM_DELIMITER)
+ || key.equals(LIBSVM_INDEX_DELIM)))
{
- String msg = "Only parameters allowed are: " + IO_FILENAME + ","
- + READROWPARAM + ","
- + READCOLPARAM;
+ String msg = "Only parameters allowed are: " + IO_FILENAME + ","
+ + READROWPARAM + "," + READCOLPARAM
+ + DELIM_DELIMITER + "," + LIBSVM_INDEX_DELIM;
raiseValidateError("Invalid parameter " + key + " in read statement: " +
toString() + ". " + msg, conditional, LanguageErrorCodes.INVALID_PARAMETERS);
}
}
}
+ // DEFAULT for "sep" : ","
+ if (getVarParam(DELIM_DELIMITER) == null) {
+ addVarParam(DELIM_DELIMITER, new StringIdentifier(DEFAULT_DELIM_DELIMITER, this));
+ }
+ else {
+ if ((getVarParam(DELIM_DELIMITER) instanceof ConstIdentifier)
+ && (!(getVarParam( DELIM_DELIMITER) instanceof StringIdentifier))) {
+ raiseValidateError( "For delimited file " + getVarParam(DELIM_DELIMITER) + " must be a string value ", conditional);
+ }
+ }
+ // DEFAULT for "indSep": ":"
+ if(getVarParam(LIBSVM_INDEX_DELIM) == null) {
+ addVarParam(LIBSVM_INDEX_DELIM, new StringIdentifier(DEFAULT_LIBSVM_INDEX_DELIM, this));
+ }
+ else {
+ if((getVarParam(LIBSVM_INDEX_DELIM) instanceof ConstIdentifier)
+ && (!(getVarParam(LIBSVM_INDEX_DELIM) instanceof StringIdentifier))) {
+ raiseValidateError(
+ "For delimited file " + getVarParam(LIBSVM_INDEX_DELIM) + " must be a string value ", conditional);
+ }
+ }
}
- dataTypeString = (getVarParam(DATATYPEPARAM) == null) ? null : getVarParam(DATATYPEPARAM).toString();
+ dataTypeString = (getVarParam(DATATYPEPARAM) == null) ? null : getVarParam(DATATYPEPARAM).toString();
if ( dataTypeString == null || dataTypeString.equalsIgnoreCase(Statement.MATRIX_DATA_TYPE)
|| dataTypeString.equalsIgnoreCase(Statement.FRAME_DATA_TYPE)) {
@@ -1203,8 +1232,8 @@ public class DataExpression extends DataIdentifier
// initialize size of target data identifier to UNKNOWN
getOutput().setDimensions(-1, -1);
- if ( !isCSV && ConfigurationManager.getCompilerConfig()
- .getBool(ConfigType.REJECT_READ_WRITE_UNKNOWNS) //skip check for csv format / jmlc api
+ if (!isCSV && !isLIBSVM && ConfigurationManager.getCompilerConfig()
+ .getBool(ConfigType.REJECT_READ_WRITE_UNKNOWNS) //skip check for csv/libsvm format / jmlc api
&& (getVarParam(READROWPARAM) == null || getVarParam(READCOLPARAM) == null) ) {
raiseValidateError("Missing or incomplete dimension information in read statement: "
+ mtdFileName, conditional, LanguageErrorCodes.INVALID_PARAMETERS);
@@ -1229,6 +1258,15 @@ public class DataExpression extends DataIdentifier
}
}
+ if(isLIBSVM) {
+ Long dim2 = (getVarParam(READCOLPARAM) == null) ? null : Long.valueOf(getVarParam(READCOLPARAM).toString());
+ if(dim2 < 0 && ConfigurationManager.getCompilerConfig()
+ .getBool(ConfigType.REJECT_READ_WRITE_UNKNOWNS)) {
+ raiseValidateError("Invalid dimension information in read statement", conditional, LanguageErrorCodes.INVALID_PARAMETERS);
+ }
+ getOutput().setDimensions(-1, dim2 + 1);
+ }
+
// initialize block dimensions to UNKNOWN
getOutput().setBlocksize(-1);
@@ -1292,7 +1330,7 @@ public class DataExpression extends DataIdentifier
case WRITE:
- // for delimited format, if no delimiter specified THEN set default ","
+ // for CSV format, if no delimiter specified THEN set default ","
if (getVarParam(FORMAT_TYPE) == null || getVarParam(FORMAT_TYPE).toString().equalsIgnoreCase(FileFormat.CSV.toString())){
if (getVarParam(DELIM_DELIMITER) == null) {
addVarParam(DELIM_DELIMITER, new StringIdentifier(DEFAULT_DELIM_DELIMITER, this));
@@ -1304,13 +1342,20 @@ public class DataExpression extends DataIdentifier
addVarParam(DELIM_SPARSE, new BooleanIdentifier(DEFAULT_DELIM_SPARSE, this));
}
}
-
- if (getVarParam(FORMAT_TYPE) == null || getVarParam(FORMAT_TYPE).toString().equalsIgnoreCase(FileFormat.LIBSVM.toString())){
- if (getVarParam(DELIM_SPARSE) == null) {
- addVarParam(DELIM_SPARSE, new BooleanIdentifier(DEFAULT_DELIM_SPARSE, this));
- }
- }
+ // for LIBSVM format, add the default separators if not specified
+ if (getVarParam(FORMAT_TYPE) == null || getVarParam(FORMAT_TYPE).toString().equalsIgnoreCase(FileFormat.LIBSVM.toString())) {
+ if(getVarParam(DELIM_DELIMITER) == null) {
+ addVarParam(DELIM_DELIMITER, new StringIdentifier(DEFAULT_DELIM_DELIMITER, this));
+ }
+ if(getVarParam(LIBSVM_INDEX_DELIM) == null) {
+ addVarParam(LIBSVM_INDEX_DELIM, new StringIdentifier(DEFAULT_LIBSVM_INDEX_DELIM, this));
+ }
+ if(getVarParam(DELIM_SPARSE) == null) {
+ addVarParam(DELIM_SPARSE, new BooleanIdentifier(DEFAULT_DELIM_SPARSE, this));
+ }
+ }
+
/* NOTE MB: disabled filename concatenation because we now support dynamic rewrite
if (getVarParam(IO_FILENAME) instanceof BinaryExpression){
BinaryExpression expr = (BinaryExpression)getVarParam(IO_FILENAME);
@@ -2307,6 +2352,17 @@ public class DataExpression extends DataIdentifier
return false;
}
+ public boolean isLIBSVMReadWithUnknownSize() {
+ Expression format = getVarParam(FORMAT_TYPE);
+ if (_opcode == DataOp.READ && format != null && format.toString().equalsIgnoreCase(FileFormat.LIBSVM.toString())) {
+ Expression rows = getVarParam(READROWPARAM);
+ Expression cols = getVarParam(READCOLPARAM);
+ return (rows == null || Long.parseLong(rows.toString()) < 0)
+ || (cols == null || Long.parseLong(cols.toString()) < 0);
+ }
+ return false;
+ }
+
public boolean isRead()
{
return (_opcode == DataOp.READ);
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/SPInstructionParser.java b/src/main/java/org/apache/sysds/runtime/instructions/SPInstructionParser.java
index 1adc79f..08a6998 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/SPInstructionParser.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/SPInstructionParser.java
@@ -86,13 +86,14 @@ import org.apache.sysds.runtime.instructions.spark.UnaryFrameSPInstruction;
import org.apache.sysds.runtime.instructions.spark.UnaryMatrixSPInstruction;
import org.apache.sysds.runtime.instructions.spark.WriteSPInstruction;
import org.apache.sysds.runtime.instructions.spark.ZipmmSPInstruction;
+import org.apache.sysds.runtime.instructions.spark.LIBSVMReblockSPInstruction;
public class SPInstructionParser extends InstructionParser
-{
+{
public static final HashMap<String, SPType> String2SPInstructionType;
static {
String2SPInstructionType = new HashMap<>();
-
+
//unary aggregate operators
String2SPInstructionType.put( "uak+" , SPType.AggregateUnary);
String2SPInstructionType.put( "uark+" , SPType.AggregateUnary);
@@ -136,7 +137,7 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "uaggouterchain", SPType.UaggOuterChain);
-
+
//ternary aggregate operators
String2SPInstructionType.put( "tak+*" , SPType.AggregateTernary);
String2SPInstructionType.put( "tack+*" , SPType.AggregateTernary);
@@ -146,18 +147,18 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "conv2d_bias_add", SPType.Dnn);
String2SPInstructionType.put( "maxpooling", SPType.Dnn);
String2SPInstructionType.put( "relu_maxpooling", SPType.Dnn);
-
+
String2SPInstructionType.put( RightIndex.OPCODE, SPType.MatrixIndexing);
String2SPInstructionType.put( LeftIndex.OPCODE, SPType.MatrixIndexing);
String2SPInstructionType.put( "mapLeftIndex", SPType.MatrixIndexing);
-
+
// Reorg Instruction Opcodes (repositioning of existing values)
String2SPInstructionType.put( "r'", SPType.Reorg);
String2SPInstructionType.put( "rev", SPType.Reorg);
String2SPInstructionType.put( "rdiag", SPType.Reorg);
String2SPInstructionType.put( "rshape", SPType.MatrixReshape);
String2SPInstructionType.put( "rsort", SPType.Reorg);
-
+
String2SPInstructionType.put( "+", SPType.Binary);
String2SPInstructionType.put( "-", SPType.Binary);
String2SPInstructionType.put( "*", SPType.Binary);
@@ -194,8 +195,8 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "map<=" , SPType.Binary);
String2SPInstructionType.put( "map==" , SPType.Binary);
String2SPInstructionType.put( "map!=" , SPType.Binary);
-
- // Boolean Instruction Opcodes
+
+ // Boolean Instruction Opcodes
String2SPInstructionType.put( "&&" , SPType.Binary);
String2SPInstructionType.put( "||" , SPType.Binary);
String2SPInstructionType.put( "xor" , SPType.Binary);
@@ -213,23 +214,24 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "mapbitwXor", SPType.Binary);
String2SPInstructionType.put( "mapbitwShiftL", SPType.Binary);
String2SPInstructionType.put( "mapbitwShiftR", SPType.Binary);
-
+
// Builtin Instruction Opcodes
String2SPInstructionType.put( "max" , SPType.Binary);
String2SPInstructionType.put( "min" , SPType.Binary);
String2SPInstructionType.put( "mapmax" , SPType.Binary);
String2SPInstructionType.put( "mapmin" , SPType.Binary);
-
- // REBLOCK Instruction Opcodes
+
+ // REBLOCK Instruction Opcodes
String2SPInstructionType.put( "rblk" , SPType.Reblock);
String2SPInstructionType.put( "csvrblk", SPType.CSVReblock);
-
+ String2SPInstructionType.put("libsvmrblk", SPType.LIBSVMReblock);
+
// Spark-specific instructions
String2SPInstructionType.put( Checkpoint.OPCODE, SPType.Checkpoint);
String2SPInstructionType.put( Compression.OPCODE, SPType.Compression);
String2SPInstructionType.put( DeCompression.OPCODE, SPType.DeCompression);
-
- // Builtin Instruction Opcodes
+
+ // Builtin Instruction Opcodes
String2SPInstructionType.put( "log" , SPType.Builtin);
String2SPInstructionType.put( "log_nz" , SPType.Builtin);
@@ -257,7 +259,7 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "isna", SPType.Unary);
String2SPInstructionType.put( "isnan", SPType.Unary);
String2SPInstructionType.put( "isinf", SPType.Unary);
-
+
// Parameterized Builtin Functions
String2SPInstructionType.put( "groupedagg", SPType.ParameterizedBuiltin);
String2SPInstructionType.put( "mapgroupedagg", SPType.ParameterizedBuiltin);
@@ -270,18 +272,18 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "transformapply", SPType.ParameterizedBuiltin);
String2SPInstructionType.put( "transformdecode",SPType.ParameterizedBuiltin);
String2SPInstructionType.put( "transformencode",SPType.MultiReturnBuiltin);
-
+
String2SPInstructionType.put( "mappend", SPType.MAppend);
String2SPInstructionType.put( "rappend", SPType.RAppend);
String2SPInstructionType.put( "gappend", SPType.GAppend);
String2SPInstructionType.put( "galignedappend", SPType.GAlignedAppend);
-
+
String2SPInstructionType.put( "cbind", SPType.BuiltinNary);
String2SPInstructionType.put( "rbind", SPType.BuiltinNary);
String2SPInstructionType.put( "nmin", SPType.BuiltinNary);
String2SPInstructionType.put( "nmax", SPType.BuiltinNary);
String2SPInstructionType.put( "n+", SPType.BuiltinNary);
-
+
String2SPInstructionType.put( DataGen.RAND_OPCODE , SPType.Rand);
String2SPInstructionType.put( DataGen.SEQ_OPCODE , SPType.Rand);
String2SPInstructionType.put( DataGen.SAMPLE_OPCODE, SPType.Rand);
@@ -290,12 +292,12 @@ public class SPInstructionParser extends InstructionParser
//ternary instruction opcodes
String2SPInstructionType.put( "ctable", SPType.Ctable);
String2SPInstructionType.put( "ctableexpand", SPType.Ctable);
-
+
//ternary instruction opcodes
String2SPInstructionType.put( "+*", SPType.Ternary);
String2SPInstructionType.put( "-*", SPType.Ternary);
String2SPInstructionType.put( "ifelse", SPType.Ternary);
-
+
//quaternary instruction opcodes
String2SPInstructionType.put( WeightedSquaredLoss.OPCODE, SPType.Quaternary);
String2SPInstructionType.put( WeightedSquaredLossR.OPCODE, SPType.Quaternary);
@@ -307,7 +309,7 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( WeightedCrossEntropyR.OPCODE,SPType.Quaternary);
String2SPInstructionType.put( WeightedUnaryMM.OPCODE, SPType.Quaternary);
String2SPInstructionType.put( WeightedUnaryMMR.OPCODE, SPType.Quaternary);
-
+
//cumsum/cumprod/cummin/cummax
String2SPInstructionType.put( "ucumack+" , SPType.CumsumAggregate);
String2SPInstructionType.put( "ucumac*" , SPType.CumsumAggregate);
@@ -325,21 +327,21 @@ public class SPInstructionParser extends InstructionParser
String2SPInstructionType.put( "cov" , SPType.Covariance);
String2SPInstructionType.put( "qsort" , SPType.QSort);
String2SPInstructionType.put( "qpick" , SPType.QPick);
-
+
String2SPInstructionType.put( "binuaggchain", SPType.BinUaggChain);
-
+
String2SPInstructionType.put( "write" , SPType.Write);
-
+
String2SPInstructionType.put( "castdtm" , SPType.Cast);
String2SPInstructionType.put( "castdtf" , SPType.Cast);
-
+
String2SPInstructionType.put( "spoof" , SPType.SpoofFused);
}
public static SPInstruction parseSingleInstruction (String str ) {
if ( str == null || str.isEmpty() )
return null;
- SPType cptype = InstructionUtils.getSPType(str);
+ SPType cptype = InstructionUtils.getSPType(str);
if ( cptype == null )
// return null;
throw new DMLRuntimeException("Invalid SP Instruction Type: " + str);
@@ -348,13 +350,13 @@ public class SPInstructionParser extends InstructionParser
throw new DMLRuntimeException("Unable to parse instruction: " + str);
return spinst;
}
-
+
public static SPInstruction parseSingleInstruction ( SPType sptype, String str ) {
- if ( str == null || str.isEmpty() )
+ if ( str == null || str.isEmpty() )
return null;
-
+
String [] parts = null;
- switch(sptype)
+ switch(sptype)
{
// matrix multiplication instructions
case CPMM:
@@ -375,48 +377,51 @@ public class SPInstructionParser extends InstructionParser
return ZipmmSPInstruction.parseInstruction(str);
case PMAPMM:
return PMapmmSPInstruction.parseInstruction(str);
-
-
+
+
case UaggOuterChain:
return UaggOuterChainSPInstruction.parseInstruction(str);
-
+
case AggregateUnary:
return AggregateUnarySPInstruction.parseInstruction(str);
-
+
case AggregateTernary:
return AggregateTernarySPInstruction.parseInstruction(str);
-
+
case Dnn:
return DnnSPInstruction.parseInstruction(str);
case MatrixIndexing:
return IndexingSPInstruction.parseInstruction(str);
-
+
case Reorg:
return ReorgSPInstruction.parseInstruction(str);
-
+
case Binary:
return BinarySPInstruction.parseInstruction(str);
case Ternary:
return TernarySPInstruction.parseInstruction(str);
-
+
//ternary instructions
case Ctable:
return CtableSPInstruction.parseInstruction(str);
-
+
//quaternary instructions
case Quaternary:
return QuaternarySPInstruction.parseInstruction(str);
-
- // Reblock instructions
+
+ // Reblock instructions
case Reblock:
return ReblockSPInstruction.parseInstruction(str);
-
+
case CSVReblock:
return CSVReblockSPInstruction.parseInstruction(str);
-
- case Builtin:
+
+ case LIBSVMReblock:
+ return LIBSVMReblockSPInstruction.parseInstruction(str);
+
+ case Builtin:
parts = InstructionUtils.getInstructionPartsWithValueType(str);
if ( parts[0].equals("log") || parts[0].equals("log_nz") ) {
if ( parts.length == 3 ) {
@@ -430,7 +435,7 @@ public class SPInstructionParser extends InstructionParser
else {
throw new DMLRuntimeException("Invalid Builtin Instruction: " + str );
}
-
+
case Unary:
parts = InstructionUtils.getInstructionPartsWithValueType(str);
CPOperand in = new CPOperand(parts[1]);
@@ -440,67 +445,67 @@ public class SPInstructionParser extends InstructionParser
return UnaryFrameSPInstruction.parseInstruction(str);
case BuiltinNary:
return BuiltinNarySPInstruction.parseInstruction(str);
-
+
case ParameterizedBuiltin:
return ParameterizedBuiltinSPInstruction.parseInstruction(str);
-
+
case MultiReturnBuiltin:
return MultiReturnParameterizedBuiltinSPInstruction.parseInstruction(str);
-
+
case MatrixReshape:
return MatrixReshapeSPInstruction.parseInstruction(str);
-
+
case MAppend: //matrix/frame
return AppendMSPInstruction.parseInstruction(str);
-
+
case RAppend: //matrix/frame
return AppendRSPInstruction.parseInstruction(str);
-
- case GAppend:
+
+ case GAppend:
return AppendGSPInstruction.parseInstruction(str);
-
+
case GAlignedAppend:
return AppendGAlignedSPInstruction.parseInstruction(str);
-
+
case Rand:
return RandSPInstruction.parseInstruction(str);
-
- case QSort:
+
+ case QSort:
return QuantileSortSPInstruction.parseInstruction(str);
-
- case QPick:
+
+ case QPick:
return QuantilePickSPInstruction.parseInstruction(str);
-
+
case Write:
return WriteSPInstruction.parseInstruction(str);
-
+
case CumsumAggregate:
return CumulativeAggregateSPInstruction.parseInstruction(str);
-
+
case CumsumOffset:
- return CumulativeOffsetSPInstruction.parseInstruction(str);
-
+ return CumulativeOffsetSPInstruction.parseInstruction(str);
+
case CentralMoment:
return CentralMomentSPInstruction.parseInstruction(str);
-
+
case Covariance:
return CovarianceSPInstruction.parseInstruction(str);
-
+
case BinUaggChain:
return BinUaggChainSPInstruction.parseInstruction(str);
-
+
case Checkpoint:
return CheckpointSPInstruction.parseInstruction(str);
-
+
case Compression:
return CompressionSPInstruction.parseInstruction(str);
case SpoofFused:
return SpoofSPInstruction.parseInstruction(str);
-
+
case Cast:
return CastSPInstruction.parseInstruction(str);
-
+
default:
throw new DMLRuntimeException("Invalid SP Instruction Type: " + sptype );
}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
index ec7c1c4..5ff1e8e 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/ParameterizedBuiltinCPInstruction.java
@@ -341,6 +341,7 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
int decimal = (getParam("decimal") != null) ? Integer.parseInt(getParam("decimal")) : TOSTRING_DECIMAL;
boolean sparse = (getParam("sparse") != null) ? Boolean.parseBoolean(getParam("sparse")) : TOSTRING_SPARSE;
String separator = (getParam("sep") != null) ? getParam("sep") : TOSTRING_SEPARATOR;
+ String indexSeparator = (getParam("indSep") != null) ? getParam("indSep") : TOSTRING_SEPARATOR;
String lineSeparator = (getParam("linesep") != null) ? getParam("linesep") : TOSTRING_LINESEPARATOR;
// get input matrix/frame and convert to string
@@ -404,9 +405,8 @@ public class ParameterizedBuiltinCPInstruction extends ComputationCPInstruction
// warn on truncation because users might not be aware and use toString for verification
if((getParam("rows") == null && data.getNumRows() > rows) ||
(getParam("cols") == null && data.getNumColumns() > cols)) {
- LOG.warn("Truncating " + data.getClass().getSimpleName() + " of size " + data.getNumRows() + "x"
- + data.getNumColumns() + " to " + rows + "x" + cols + ". "
- + "Use toString(X, rows=..., cols=...) if necessary.");
+ LOG.warn("Truncating " + data.getClass().getSimpleName() + " of size " + data.getNumRows() + "x" + data
+ .getNumColumns() + " to " + rows + "x" + cols + ". " + "Use toString(X, rows=..., cols=...) if necessary.");
}
}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/cp/VariableCPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/cp/VariableCPInstruction.java
index 52c6c74..7ca7ef2 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/cp/VariableCPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/cp/VariableCPInstruction.java
@@ -50,9 +50,11 @@ import org.apache.sysds.runtime.instructions.Instruction;
import org.apache.sysds.runtime.instructions.InstructionUtils;
import org.apache.sysds.runtime.io.FileFormatProperties;
import org.apache.sysds.runtime.io.FileFormatPropertiesCSV;
+import org.apache.sysds.runtime.io.FileFormatPropertiesLIBSVM;
import org.apache.sysds.runtime.io.IOUtilFunctions;
import org.apache.sysds.runtime.io.WriterMatrixMarket;
import org.apache.sysds.runtime.io.WriterTextCSV;
+import org.apache.sysds.runtime.io.WriterTextLIBSVM;
import org.apache.sysds.runtime.lineage.LineageItem;
import org.apache.sysds.runtime.lineage.LineageItemUtils;
import org.apache.sysds.runtime.lineage.LineageTraceable;
@@ -105,20 +107,20 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
Read,
SetFileName,
}
-
+
private static final IDSequence _uniqueVarID = new IDSequence(true);
private static final int CREATEVAR_FILE_NAME_VAR_POS=3;
-
+
private final VariableOperationCode opcode;
private final List<CPOperand> inputs;
private final CPOperand output;
private final MetaData metadata;
private final UpdateType _updateType;
private final boolean _containsPreadPrefix;
-
+
// Frame related members
private final String _schema;
-
+
// CSV and LIBSVM related members (used only in createvar instructions)
private final FileFormatProperties _formatProperties;
@@ -138,7 +140,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
_containsPreadPrefix = in1 != null && in1.getName()
.contains(org.apache.sysds.lops.Data.PREAD_PREFIX);
}
-
+
private VariableCPInstruction(VariableOperationCode op, CPOperand in1, CPOperand in2, CPOperand in3, CPOperand out,
String sopcode, String istr) {
this(op, in1, in2, in3, out, null, null, null, null, sopcode, istr);
@@ -158,56 +160,56 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
}
private static VariableOperationCode getVariableOperationCode ( String str ) {
-
+
if ( str.equalsIgnoreCase("createvar"))
return VariableOperationCode.CreateVariable;
-
+
else if ( str.equalsIgnoreCase("assignvar"))
return VariableOperationCode.AssignVariable;
-
+
else if ( str.equalsIgnoreCase("cpvar"))
return VariableOperationCode.CopyVariable;
-
+
else if ( str.equalsIgnoreCase("mvvar"))
return VariableOperationCode.MoveVariable;
-
+
else if ( str.equalsIgnoreCase("rmvar") )
return VariableOperationCode.RemoveVariable;
-
+
else if ( str.equalsIgnoreCase("rmfilevar") )
return VariableOperationCode.RemoveVariableAndFile;
-
+
else if ( str.equalsIgnoreCase(UnaryCP.CAST_AS_SCALAR_OPCODE) )
return VariableOperationCode.CastAsScalarVariable;
-
+
else if ( str.equalsIgnoreCase(UnaryCP.CAST_AS_MATRIX_OPCODE) )
return VariableOperationCode.CastAsMatrixVariable;
-
+
else if ( str.equalsIgnoreCase(UnaryCP.CAST_AS_FRAME_OPCODE) )
return VariableOperationCode.CastAsFrameVariable;
-
+
else if ( str.equalsIgnoreCase(UnaryCP.CAST_AS_DOUBLE_OPCODE) )
return VariableOperationCode.CastAsDoubleVariable;
-
+
else if ( str.equalsIgnoreCase(UnaryCP.CAST_AS_INT_OPCODE) )
return VariableOperationCode.CastAsIntegerVariable;
-
+
else if ( str.equalsIgnoreCase(UnaryCP.CAST_AS_BOOLEAN_OPCODE) )
return VariableOperationCode.CastAsBooleanVariable;
-
+
else if ( str.equalsIgnoreCase("write") )
return VariableOperationCode.Write;
-
+
else if ( str.equalsIgnoreCase("read") )
return VariableOperationCode.Read;
-
+
else if ( str.equalsIgnoreCase("setfilename") )
return VariableOperationCode.SetFileName;
-
+
else
throw new DMLRuntimeException("Invalid function: " + str);
}
-
+
/**
* Checks if this instruction is a remove instruction for varName
*
@@ -222,11 +224,11 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
}
return false;
}
-
+
public boolean isRemoveVariableNoFile() {
return opcode == VariableOperationCode.RemoveVariable;
}
-
+
public boolean isRemoveVariable() {
return opcode == VariableOperationCode.RemoveVariable
|| opcode == VariableOperationCode.RemoveVariableAndFile;
@@ -235,12 +237,12 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
public boolean isAssignVariable() {
return opcode == VariableOperationCode.AssignVariable;
}
-
+
public boolean isAssignOrCopyVariable() {
return opcode == VariableOperationCode.AssignVariable
|| opcode == VariableOperationCode.CopyVariable;
}
-
+
public boolean isCreateVariable() {
return opcode == VariableOperationCode.CreateVariable;
}
@@ -252,38 +254,38 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
public FileFormatProperties getFormatProperties() {
return _formatProperties;
}
-
+
public List<CPOperand> getInputs() {
return inputs;
}
-
+
public CPOperand getInput1() {
return getInput(0);
}
-
+
public CPOperand getInput2() {
return getInput(1);
}
-
+
public CPOperand getInput3() {
return getInput(2);
}
-
+
public CPOperand getInput4() {
return getInput(3);
}
-
+
public CPOperand getInput(int index) {
if( inputs.size() <= index )
return null;
return inputs.get(index);
}
-
+
public void addInput(CPOperand input) {
if( input != null )
inputs.add(input);
}
-
+
public String getOutputVariableName(){
String ret = null;
if( output != null )
@@ -304,12 +306,12 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
return 2;
}
}
-
+
public static VariableCPInstruction parseInstruction ( String str ) {
String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );
String opcode = parts[0];
VariableOperationCode voc = getVariableOperationCode(opcode);
-
+
if ( voc == VariableOperationCode.CreateVariable ){
if ( parts.length < 5 ) //&& parts.length != 10 )
throw new DMLRuntimeException("Invalid number of operands in createvar instruction: " + str);
@@ -331,11 +333,11 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
if( voc != VariableOperationCode.RemoveVariable )
InstructionUtils.checkNumFields ( parts, getArity(voc) ); // no output
}
-
+
CPOperand in1=null, in2=null, in3=null, in4=null, out=null;
-
+
switch (voc) {
-
+
case CreateVariable:
// variable name
DataType dt = DataType.valueOf(parts[4]);
@@ -347,7 +349,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
in2 = new CPOperand(parts[2], ValueType.STRING, DataType.SCALAR);
// file name override flag (always literal)
in3 = new CPOperand(parts[3], ValueType.BOOLEAN, DataType.SCALAR);
-
+
// format
String fmt = parts[5];
if ( fmt.equalsIgnoreCase("csv") ) {
@@ -356,6 +358,13 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
// 14 inputs: createvar corresponding to READ -- includes properties hasHeader, delim, fill, and fillValue
if ( parts.length < 14+extSchema || parts.length > 16+extSchema )
throw new DMLRuntimeException("Invalid number of operands in createvar instruction: " + str);
+ }
+ else if(fmt.equalsIgnoreCase("libsvm")) {
+ // 13 inputs: createvar corresponding to WRITE -- includes properties delim, index delim, and sparse
+ // 12 inputs: createvar corresponding to READ -- includes properties delim, index delim, and sparse
+
+ if(parts.length < 12 + extSchema)
+ throw new DMLRuntimeException("Invalid number of operands in createvar instruction: " + str);
}
else {
if ( parts.length != 6 && parts.length != 11+extSchema )
@@ -398,10 +407,10 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
UpdateType updateType = UpdateType.COPY;
if ( parts.length >= 11 )
updateType = UpdateType.valueOf(parts[10].toUpperCase());
-
+
//handle frame schema
String schema = (dt==DataType.FRAME && parts.length>=12) ? parts[parts.length-1] : null;
-
+
if ( fmt.equalsIgnoreCase("csv") ) {
// Cretevar instructions for CSV format either has 13 or 14 inputs.
// 13 inputs: createvar corresponding to WRITE -- includes properties hasHeader, delim, and sparse
@@ -424,7 +433,29 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
naStrings = parts[curPos+4];
fmtProperties = new FileFormatPropertiesCSV(hasHeader, delim, fill, fillValue, naStrings) ;
}
- return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType, fmtProperties, schema, opcode, str);
+ return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType,
+ fmtProperties, schema, opcode, str);
+ }
+ else if(fmt.equalsIgnoreCase("libsvm")) {
+ // Cretevar instructions for LIBSVM format has 13.
+ // 13 inputs: createvar corresponding to WRITE -- includes properties delim, index delim and sparse
+ // 12 inputs: createvar corresponding to READ -- includes properties delim, index delim, and sparse
+ FileFormatProperties fmtProperties = null;
+ int curPos = 11;
+ if(parts.length == 12 + extSchema) {
+ String delim = parts[curPos];
+ String indexDelim = parts[curPos + 1];
+ fmtProperties = new FileFormatPropertiesLIBSVM(delim, indexDelim);
+ }
+ else {
+ String delim = parts[curPos];
+ String indexDelim = parts[curPos + 1];
+ boolean sparse = Boolean.parseBoolean(parts[curPos + 2]);
+ fmtProperties = new FileFormatPropertiesLIBSVM(delim, indexDelim, sparse);
+ }
+
+ return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType,
+ fmtProperties, schema, opcode, str);
}
else {
return new VariableCPInstruction(VariableOperationCode.CreateVariable, in1, in2, in3, iimd, updateType, schema, opcode, str);
@@ -433,28 +464,28 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
in1 = new CPOperand(parts[1]);
in2 = new CPOperand(parts[2]);
break;
-
+
case CopyVariable:
// Value types are not given here
boolean withTypes = parts[1].split(VALUETYPE_PREFIX).length > 2 && parts[2].split(VALUETYPE_PREFIX).length > 2;
in1 = withTypes ? new CPOperand(parts[1]) : new CPOperand(parts[1], ValueType.UNKNOWN, DataType.UNKNOWN);
in2 = withTypes ? new CPOperand(parts[2]) : new CPOperand(parts[2], ValueType.UNKNOWN, DataType.UNKNOWN);
break;
-
+
case MoveVariable:
in1 = new CPOperand(parts[1], ValueType.UNKNOWN, DataType.UNKNOWN);
in2 = new CPOperand(parts[2], ValueType.UNKNOWN, DataType.UNKNOWN);
if(parts.length > 3)
in3 = new CPOperand(parts[3], ValueType.UNKNOWN, DataType.UNKNOWN);
break;
-
+
case RemoveVariable:
VariableCPInstruction rminst = new VariableCPInstruction(
getVariableOperationCode(opcode), null, null, null, out, opcode, str);
for( int i=1; i<parts.length; i++ )
rminst.addInput(new CPOperand(parts[i], ValueType.UNKNOWN, DataType.SCALAR));
return rminst;
-
+
case RemoveVariableAndFile:
in1 = new CPOperand(parts[1]);
in2 = new CPOperand(parts[2]);
@@ -462,7 +493,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
if ( in2.getValueType() != ValueType.BOOLEAN)
throw new DMLRuntimeException("Unexpected value type for second argument in: " + str);
break;
-
+
case CastAsScalarVariable:
case CastAsMatrixVariable:
case CastAsFrameVariable:
@@ -472,12 +503,12 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
in1 = new CPOperand(parts[1]); // first operand is a variable name => string value type
out = new CPOperand(parts[2]); // output variable name
break;
-
+
case Write:
in1 = new CPOperand(parts[1]);
in2 = new CPOperand(parts[2]);
in3 = new CPOperand(parts[3]);
-
+
FileFormatProperties fprops = null;
if ( in3.getName().equalsIgnoreCase("csv") ) {
boolean hasHeader = Boolean.parseBoolean(parts[4]);
@@ -485,10 +516,13 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
boolean sparse = Boolean.parseBoolean(parts[6]);
fprops = new FileFormatPropertiesCSV(hasHeader, delim, sparse);
in4 = new CPOperand(parts[7]); // description
- }
+ }
else if ( in3.getName().equalsIgnoreCase("libsvm") ) {
- fprops = new FileFormatProperties();
- }
+ String delim = parts[4];
+ String indexDelim = parts[5];
+ boolean sparse = Boolean.parseBoolean(parts[6]);
+ fprops = new FileFormatPropertiesLIBSVM(delim, indexDelim, sparse);
+ }
else {
fprops = new FileFormatProperties();
in4 = new CPOperand(parts[4]); // description
@@ -496,24 +530,24 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
VariableCPInstruction inst = new VariableCPInstruction(
getVariableOperationCode(opcode), in1, in2, in3, out, null, fprops, null, null, opcode, str);
inst.addInput(in4);
-
+
return inst;
-
+
case Read:
in1 = new CPOperand(parts[1]);
in2 = new CPOperand(parts[2]);
break;
-
+
case SetFileName:
in1 = new CPOperand(parts[1]); // variable name
in2 = new CPOperand(parts[2], ValueType.UNKNOWN, DataType.UNKNOWN); // file name
in3 = new CPOperand(parts[3], ValueType.UNKNOWN, DataType.UNKNOWN); // option: remote or local
break;
-
+
}
return new VariableCPInstruction(getVariableOperationCode(opcode), in1, in2, in3, out, opcode, str);
}
-
+
@Override
public void processInstruction(ExecutionContext ec) {
switch ( opcode )
@@ -521,29 +555,29 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
case CreateVariable:
processCreateVariableInstruction(ec);
break;
-
+
case AssignVariable:
// assign value of variable to the other
ec.setScalarOutput(getInput2().getName(), ec.getScalarInput(getInput1()));
break;
-
+
case CopyVariable:
processCopyInstruction(ec);
break;
-
+
case MoveVariable:
processMoveInstruction(ec);
break;
-
+
case RemoveVariable:
for( CPOperand input : inputs )
processRmvarInstruction(ec, input.getName());
break;
-
+
case RemoveVariableAndFile:
processRemoveVariableAndFileInstruction(ec);
break;
-
+
case CastAsScalarVariable: //castAsScalarVariable
processCastAsScalarVariableInstruction(ec);
break;
@@ -555,7 +589,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
case CastAsFrameVariable:
processCastAsFrameVariableInstruction(ec);
break;
-
+
case CastAsDoubleVariable:
ScalarObject scalarDoubleInput = ec.getScalarInput(getInput1());
ec.setScalarOutput(output.getName(), ScalarObjectFactory.castToDouble(scalarDoubleInput));
@@ -570,19 +604,19 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
ScalarObject scalarBooleanInput = ec.getScalarInput(getInput1());
ec.setScalarOutput(output.getName(), new BooleanObject(scalarBooleanInput.getBooleanValue()));
break;
-
+
case Read:
processReadInstruction(ec);
break;
-
+
case Write:
processWriteInstruction(ec);
break;
-
+
case SetFileName:
processSetFileNameInstruction(ec);
break;
-
+
default:
throw new DMLRuntimeException("Unknown opcode: " + opcode );
}
@@ -590,6 +624,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
/**
* Handler for processInstruction "CreateVariable" case
+ *
* @param ec execution context of the instruction
*/
private void processCreateVariableInstruction(ExecutionContext ec){
@@ -597,7 +632,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
//would cause a buffer pool memory leak as these objects would never be removed
if(ec.containsVariable(getInput1()))
processRmvarInstruction(ec, getInput1().getName());
-
+
switch(getInput1().getDataType()) {
case MATRIX: {
String fname = createUniqueFilename();
@@ -656,7 +691,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
.startsWith(org.apache.sysds.lops.Data.PREAD_PREFIX));
obj.setFileFormatProperties(_formatProperties);
}
-
+
/**
* Handler for mvvar instructions.
* Example: mvvar <srcvar> <destFile> <format>
@@ -667,18 +702,18 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
*/
@SuppressWarnings("rawtypes")
private void processMoveInstruction(ExecutionContext ec) {
-
+
if ( getInput3() == null ) {
// example: mvvar tempA A (note that mvvar does not carry the data types)
-
- // get and remove source variable
+
+ // get and remove source variable
Data srcData = ec.removeVariable(getInput1().getName());
-
+
if ( srcData == null ) {
throw new DMLRuntimeException("Unexpected error: could not find a data object "
+ "for variable name:" + getInput1().getName() + ", while processing instruction ");
}
-
+
// remove existing variable bound to target name and
// cleanup matrix/frame/list data if necessary
if( srcData.getDataType().isMatrix() || srcData.getDataType().isFrame() ) {
@@ -686,7 +721,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
if( tgtData != null && srcData != tgtData )
ec.cleanupDataObject(tgtData);
}
-
+
// do the actual move
ec.setVariable(getInput2().getName(), srcData);
}
@@ -694,9 +729,9 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
// example instruction: mvvar <srcVar> <destFile> <format>
if ( ec.getVariable(getInput1().getName()) == null )
throw new DMLRuntimeException("Unexpected error: could not find a data object for variable name:" + getInput1().getName() + ", while processing instruction " +this.toString());
-
+
Data object = ec.getVariable(getInput1().getName());
-
+
if ( getInput3().getName().equalsIgnoreCase("binaryblock") ) {
boolean success = false;
success = ((CacheableData)object).moveData(getInput2().getName(), getInput3().getName());
@@ -716,14 +751,14 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
/**
* Handler for RemoveVariableAndFile instruction
- *
+ *
* @param ec execution context
*/
private void processRemoveVariableAndFileInstruction(ExecutionContext ec){
// Remove the variable from HashMap _variables, and possibly delete the data on disk.
boolean del = ( (BooleanObject) ec.getScalarInput(getInput2().getName(), getInput2().getValueType(), true) ).getBooleanValue();
MatrixObject m = (MatrixObject) ec.removeVariable(getInput1().getName());
-
+
if ( !del ) {
// HDFS file should be retailed after clearData(),
// therefore data must be exported if dirty flag is set
@@ -735,11 +770,11 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
//cleanDataOnHDFS(pb, input1.getName());
cleanDataOnHDFS( m );
}
-
+
// check if in-memory object can be cleaned up
if ( !ec.getVariables().hasReferences(m) ) {
// no other variable in the symbol table points to the same Data object as that of input1.getName()
-
+
//remove matrix object from cache
m.clearData(ec.getTID());
}
@@ -747,6 +782,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
/**
* Process CastAsScalarVariable instruction.
+ *
* @param ec execution context
*/
private void processCastAsScalarVariableInstruction(ExecutionContext ec){
@@ -795,6 +831,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
/**
* Handler for CastAsMatrixVariable instruction
+ *
* @param ec execution context
*/
private void processCastAsMatrixVariableInstruction(ExecutionContext ec) {
@@ -845,6 +882,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
/**
* Handler for CastAsFrameVariable instruction
+ *
* @param ec execution context
*/
private void processCastAsFrameVariableInstruction(ExecutionContext ec){
@@ -871,6 +909,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
/**
* Handler for Read instruction
+ *
* @param ec execution context
*/
private void processReadInstruction(ExecutionContext ec){
@@ -890,7 +929,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
res = new StringObject(HDFSTool.readStringFromHDFSFile(getInput2().getName()));
break;
default:
- throw new DMLRuntimeException("Invalid value type ("
+ throw new DMLRuntimeException("Invalid value type ("
+ getInput1().getValueType() + ") while processing readScalar instruction.");
}
} catch ( IOException e ) {
@@ -898,7 +937,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
}
ec.setScalarOutput(getInput1().getName(), res);
}
-
+
/**
* Handler for cpvar instructions.
* Example: cpvar <srcvar> <destvar>
@@ -908,21 +947,21 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
private void processCopyInstruction(ExecutionContext ec) {
// get source variable
Data dd = ec.getVariable(getInput1().getName());
-
+
if ( dd == null )
throw new DMLRuntimeException("Unexpected error: could not find a data object for variable name:" + getInput1().getName() + ", while processing instruction " +this.toString());
-
+
// remove existing variable bound to target name
Data input2_data = ec.removeVariable(getInput2().getName());
-
+
//cleanup matrix data on fs/hdfs (if necessary)
if( input2_data != null )
ec.cleanupDataObject(input2_data);
-
+
// do the actual copy!
ec.setVariable(getInput2().getName(), dd);
}
-
+
/**
* Handler for write instructions.
*
@@ -941,7 +980,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
String desc = ec.getScalarInput(getInput4().getName(), ValueType.STRING, getInput4().isLiteral()).getStringValue();
_formatProperties.setDescription(desc);
}
-
+
if( getInput1().getDataType() == DataType.SCALAR ) {
writeScalarToHDFS(ec, fname);
}
@@ -950,6 +989,8 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
writeMMFile(ec, fname);
else if( fmt == FileFormat.CSV )
writeCSVFile(ec, fname);
+ else if(fmt == FileFormat.LIBSVM)
+ writeLIBSVMFile(ec, fname);
else {
// Default behavior
MatrixObject mo = ec.getMatrixObject(getInput1().getName());
@@ -981,11 +1022,13 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
if ( getInput3().getName().equalsIgnoreCase("remote") )
((MatrixObject)data).setFileName(getInput2().getName());
else
- throw new DMLRuntimeException("Invalid location (" + getInput3().getName() + ") in SetFileName instruction: " + instString);
- } else
+ throw new DMLRuntimeException(
+ "Invalid location (" + getInput3().getName() + ") in SetFileName instruction: " + instString);
+ }
+ else
throw new DMLRuntimeException("Invalid data type (" + getInput1().getDataType() + ") in SetFileName instruction: " + instString);
}
-
+
/**
* Remove variable instruction externalized as a static function in order to allow various
* cleanup procedures to use the same codepath as the actual rmVar instruction
@@ -1000,7 +1043,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
if( dat != null )
ec.cleanupDataObject(dat);
}
-
+
/**
* Helper function to write CSV files to HDFS.
*
@@ -1010,7 +1053,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
private void writeCSVFile(ExecutionContext ec, String fname) {
MatrixObject mo = ec.getMatrixObject(getInput1().getName());
String outFmt = "csv";
-
+
if(mo.isDirty()) {
// there exist data computed in CP that is not backed up on HDFS
// i.e., it is either in-memory or in evicted space
@@ -1020,7 +1063,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
try {
FileFormat fmt = ((MetaDataFormat)mo.getMetaData()).getFileFormat();
DataCharacteristics dc = (mo.getMetaData()).getDataCharacteristics();
- if( fmt == FileFormat.CSV
+ if( fmt == FileFormat.CSV
&& !getInput1().getName().startsWith(org.apache.sysds.lops.Data.PREAD_PREFIX) )
{
WriterTextCSV writer = new WriterTextCSV((FileFormatPropertiesCSV)_formatProperties);
@@ -1029,14 +1072,49 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
else {
mo.exportData(fname, outFmt, _formatProperties);
}
- HDFSTool.writeMetaDataFile (fname + ".mtd", mo.getValueType(), dc, FileFormat.CSV, _formatProperties, mo.getPrivacyConstraint());
+ HDFSTool.writeMetaDataFile(fname + ".mtd", mo.getValueType(), dc, FileFormat.CSV, _formatProperties,
+ mo.getPrivacyConstraint());
+ }
+ catch(IOException e) {
+ throw new DMLRuntimeException(e);
+ }
+ }
+ }
+
+ /**
+ * Helper function to write LIBSVM files to HDFS.
+ *
+ * @param ec execution context
+ * @param fname file name
+ */
+ private void writeLIBSVMFile(ExecutionContext ec, String fname) {
+ MatrixObject mo = ec.getMatrixObject(getInput1().getName());
+ String outFmt = "libsvm";
+
+ if(mo.isDirty()) {
+ // there exist data computed in CP that is not backed up on HDFS
+ // i.e., it is either in-memory or in evicted space
+ mo.exportData(fname, outFmt, _formatProperties);
+ }
+ else {
+ try {
+ FileFormat fmt = ((MetaDataFormat) mo.getMetaData()).getFileFormat();
+ DataCharacteristics dc = (mo.getMetaData()).getDataCharacteristics();
+ if(fmt == FileFormat.LIBSVM && !getInput1().getName().startsWith(org.apache.sysds.lops.Data.PREAD_PREFIX)) {
+ WriterTextLIBSVM writer = new WriterTextLIBSVM((FileFormatPropertiesLIBSVM) _formatProperties);
+ }
+ else {
+ mo.exportData(fname, outFmt, _formatProperties);
+ }
+ HDFSTool.writeMetaDataFile(fname + ".mtd", mo.getValueType(), dc, FileFormat.LIBSVM, _formatProperties,
+ mo.getPrivacyConstraint());
}
catch (IOException e) {
throw new DMLRuntimeException(e);
}
}
}
-
+
/**
* Helper function to write MM files to HDFS.
*
@@ -1055,7 +1133,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
try {
FileFormat fmt = ((MetaDataFormat)mo.getMetaData()).getFileFormat();
DataCharacteristics dc = mo.getDataCharacteristics();
- if( fmt == FileFormat.TEXT
+ if( fmt == FileFormat.TEXT
&& !getInput1().getName().startsWith(org.apache.sysds.lops.Data.PREAD_PREFIX) )
{
WriterMatrixMarket.mergeTextcellToMatrixMarket(mo.getFileName(),
@@ -1070,7 +1148,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
}
}
}
-
+
/**
* Helper function to write scalars to HDFS based on its value type.
*
@@ -1093,7 +1171,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
throw new DMLRuntimeException(e);
}
}
-
+
private static void cleanDataOnHDFS(MatrixObject mo) {
try {
String fpath = mo.getFileName();
@@ -1105,13 +1183,13 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
throw new DMLRuntimeException(e);
}
}
-
+
public static Instruction prepareRemoveInstruction(long... varName) {
String[] tmp = new String[varName.length];
Arrays.setAll(tmp, i -> String.valueOf(varName[i]));
return prepareRemoveInstruction(tmp);
}
-
+
public static Instruction prepareRemoveInstruction(String... varNames) {
StringBuilder sb = new StringBuilder();
sb.append("CP");
@@ -1123,7 +1201,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
}
return parseInstruction(sb.toString());
}
-
+
public static Instruction prepareCopyInstruction(String srcVar, String destVar) {
StringBuilder sb = new StringBuilder();
sb.append("CP");
@@ -1135,7 +1213,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
sb.append(destVar);
return parseInstruction(sb.toString());
}
-
+
public static Instruction prepMoveInstruction(String srcVar, String destFileName, String format) {
StringBuilder sb = new StringBuilder();
sb.append("CP");
@@ -1150,9 +1228,9 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
String str = sb.toString();
return parseInstruction(str);
}
-
+
public static Instruction prepMoveInstruction(String srcVar, String destVar) {
- // example: mvvar tempA A
+ // example: mvvar tempA A
StringBuilder sb = new StringBuilder();
sb.append("CP");
sb.append(Lop.OPERAND_DELIMITOR);
@@ -1164,13 +1242,13 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
String str = sb.toString();
return parseInstruction(str);
}
-
+
private static String getBasicCreatevarString(String varName, String fileName, boolean fNameOverride, DataType dt, String format) {
- //note: the filename override property leads to concatenation of unique ids in order to
+ //note: the filename override property leads to concatenation of unique ids in order to
//ensure conflicting filenames for objects that originate from the same instruction
boolean lfNameOverride = fNameOverride && !ConfigurationManager
.getCompilerConfigFlag(ConfigType.IGNORE_TEMPORARY_FILENAMES);
-
+
StringBuilder sb = new StringBuilder();
sb.append("CP");
sb.append(Lop.OPERAND_DELIMITOR);
@@ -1188,7 +1266,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
sb.append(format);
return sb.toString();
}
-
+
public static Instruction prepCreatevarInstruction(String varName, String fileName, boolean fNameOverride, String format) {
return parseInstruction(getBasicCreatevarString(varName, fileName, fNameOverride, DataType.MATRIX, format));
}
@@ -1196,7 +1274,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
public static Instruction prepCreatevarInstruction(String varName, String fileName, boolean fNameOverride, DataType dt, String format, DataCharacteristics mc, UpdateType update) {
StringBuilder sb = new StringBuilder();
sb.append(getBasicCreatevarString(varName, fileName, fNameOverride, dt, format));
-
+
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(mc.getRows());
sb.append(Lop.OPERAND_DELIMITOR);
@@ -1207,14 +1285,14 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
sb.append(mc.getNonZeros());
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(update.toString().toLowerCase());
-
+
return parseInstruction(sb.toString());
}
-
+
public static Instruction prepCreatevarInstruction(String varName, String fileName, boolean fNameOverride, DataType dt, String format, DataCharacteristics mc, UpdateType update, boolean hasHeader, String delim, boolean sparse) {
StringBuilder sb = new StringBuilder();
sb.append(getBasicCreatevarString(varName, fileName, fNameOverride, dt, format));
-
+
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(mc.getRows());
sb.append(Lop.OPERAND_DELIMITOR);
@@ -1225,18 +1303,18 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
sb.append(mc.getNonZeros());
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(update.toString().toLowerCase());
-
+
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(hasHeader);
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(delim);
sb.append(Lop.OPERAND_DELIMITOR);
sb.append(sparse);
-
+
String str = sb.toString();
return parseInstruction(str);
}
-
+
@Override
public void updateInstructionThreadID(String pattern, String replace) {
if( opcode == VariableOperationCode.CreateVariable
@@ -1249,17 +1327,17 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
int iPos = StringUtils.ordinalIndexOf(instString, Lop.OPERAND_DELIMITOR, CREATEVAR_FILE_NAME_VAR_POS);
// Find a end position of file name string.
int iPos2 = StringUtils.indexOf(instString, Lop.OPERAND_DELIMITOR, iPos+1);
-
+
StringBuilder sb = new StringBuilder();
sb.append(instString.substring(0,iPos+1)); // It takes first part before file name.
// This will replace 'pattern' with 'replace' string from file name.
sb.append(ProgramConverter.saveReplaceFilenameThreadID(instString.substring(iPos+1, iPos2+1), pattern, replace));
sb.append(instString.substring(iPos2+1)); // It takes last part after file name.
-
+
instString = sb.toString();
}
}
-
+
@Override
public Pair<String,LineageItem> getLineageItem(ExecutionContext ec) {
String varname = null;
@@ -1268,7 +1346,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
case CreateVariable:
if (!_containsPreadPrefix)
break; //otherwise fall through
-
+
case Read: {
varname = getInput1().getName();
li = new LineageItem(toString().replace(getInput1().getName(),
@@ -1312,10 +1390,10 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
case MoveVariable:
default:
}
-
+
return (li == null) ? null : Pair.of(varname, li);
}
-
+
public boolean isVariableCastInstruction() {
return opcode == VariableOperationCode.CastAsScalarVariable
|| opcode == VariableOperationCode.CastAsMatrixVariable
@@ -1324,7 +1402,7 @@ public class VariableCPInstruction extends CPInstruction implements LineageTrace
|| opcode == VariableOperationCode.CastAsDoubleVariable
|| opcode == VariableOperationCode.CastAsBooleanVariable;
}
-
+
public static String getUniqueFileName(String fname) {
return InstructionUtils.concatStrings(fname, "_", String.valueOf(_uniqueVarID.getNextID()));
}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/LIBSVMReblockSPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/LIBSVMReblockSPInstruction.java
new file mode 100644
index 0000000..f6cc0b2
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/LIBSVMReblockSPInstruction.java
@@ -0,0 +1,135 @@
+/*
+ * 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.instructions.spark;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.sysds.common.Types.DataType;
+import org.apache.sysds.common.Types.FileFormat;
+import org.apache.sysds.common.Types.ValueType;
+import org.apache.sysds.hops.recompile.Recompiler;
+import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.controlprogram.caching.CacheableData;
+import org.apache.sysds.runtime.controlprogram.caching.FrameObject;
+import org.apache.sysds.runtime.controlprogram.context.ExecutionContext;
+import org.apache.sysds.runtime.controlprogram.context.SparkExecutionContext;
+import org.apache.sysds.runtime.instructions.InstructionUtils;
+import org.apache.sysds.runtime.instructions.cp.CPOperand;
+import org.apache.sysds.runtime.instructions.spark.utils.RDDConverterUtils;
+import org.apache.sysds.runtime.matrix.data.FrameBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixBlock;
+import org.apache.sysds.runtime.matrix.data.MatrixIndexes;
+import org.apache.sysds.runtime.matrix.operators.Operator;
+import org.apache.sysds.runtime.meta.DataCharacteristics;
+import org.apache.sysds.runtime.meta.MetaDataFormat;
+import org.apache.sysds.utils.Statistics;
+
+public class LIBSVMReblockSPInstruction extends UnarySPInstruction {
+ private int _blen;
+ private String _delim;
+ private String _indexDelim;
+
+ protected LIBSVMReblockSPInstruction(Operator op, CPOperand in, CPOperand out, int br, int bc, String opcode,
+ String delim, String indexDelim, String instr) {
+ super(SPType.LIBSVMReblock, op, in, out, opcode, instr);
+ _blen = br;
+ _blen = bc;
+ _delim = delim;
+ _indexDelim = indexDelim;
+ }
+
+ public static LIBSVMReblockSPInstruction parseInstruction(String str) {
+ String opcode = InstructionUtils.getOpCode(str);
+ if(!opcode.equals("libsvmrblk"))
+ throw new DMLRuntimeException("Incorrect opcode for LIBSVMReblockSPInstruction:" + opcode);
+
+ String parts[] = InstructionUtils.getInstructionPartsWithValueType(str);
+
+ CPOperand in = new CPOperand(parts[1]);
+ CPOperand out = new CPOperand(parts[2]);
+ int blen = Integer.parseInt(parts[3]);
+ String delim = parts[4];
+ String indexDelim = parts[5];
+
+ return new LIBSVMReblockSPInstruction(null, in, out, blen, blen, opcode, delim, indexDelim, str);
+ }
+
+ @Override
+ public void processInstruction(ExecutionContext ec) {
+ SparkExecutionContext sec = (SparkExecutionContext) ec;
+
+ //sanity check input info
+ CacheableData<?> obj = sec.getCacheableData(input1.getName());
+ MetaDataFormat iimd = (MetaDataFormat) obj.getMetaData();
+ if(iimd.getFileFormat() != FileFormat.LIBSVM) {
+ throw new DMLRuntimeException(
+ "The given format is not implemented for " + "LIBSVMReblockSPInstruction:" + iimd.getFileFormat()
+ .toString());
+ }
+
+ //set output characteristics
+ DataCharacteristics mcIn = sec.getDataCharacteristics(input1.getName());
+ DataCharacteristics mcOut = sec.getDataCharacteristics(output.getName());
+ mcOut.set(mcIn.getRows(), mcIn.getCols(), _blen);
+
+ //check for in-memory reblock (w/ lazy spark context, potential for latency reduction)
+ if(Recompiler.checkCPReblock(sec, input1.getName())) {
+ if(input1.getDataType().isMatrix() || input1.getDataType().isFrame())
+ Recompiler.executeInMemoryReblock(sec, input1.getName(), output.getName());
+ Statistics.decrementNoOfExecutedSPInst();
+ return;
+ }
+
+ //execute matrix/frame libsvmreblock
+ JavaPairRDD<?, ?> out = null;
+ if(input1.getDataType() == DataType.MATRIX)
+ out = processMatrixLIBSVMReblockInstruction(sec, mcOut);
+ else if(input1.getDataType() == DataType.FRAME)
+ out = processFrameLIBSVMReblockInstruction(sec, mcOut, ((FrameObject) obj).getSchema());
+
+ // put output RDD handle into symbol table
+ sec.setRDDHandleForVariable(output.getName(), out);
+ sec.addLineageRDD(output.getName(), input1.getName());
+ }
+
+ @SuppressWarnings("unchecked")
+ protected JavaPairRDD<MatrixIndexes, MatrixBlock> processMatrixLIBSVMReblockInstruction(
+ SparkExecutionContext sec, DataCharacteristics mcOut) {
+ //get input rdd (needs to be longwritable/text for consistency with meta data, in case of
+ //serialization issues create longwritableser/textser as serializable wrappers
+ JavaPairRDD<LongWritable, Text> in = (JavaPairRDD<LongWritable, Text>) sec
+ .getRDDHandleForMatrixObject(sec.getMatrixObject(input1), FileFormat.LIBSVM);
+
+ //reblock libsvm to binary block
+ return RDDConverterUtils.libsvmToBinaryBlock(sec.getSparkContext(), in, mcOut, _delim, _indexDelim);
+ }
+
+ @SuppressWarnings({"unchecked", "unused"})
+ protected JavaPairRDD<Long, FrameBlock> processFrameLIBSVMReblockInstruction(
+ SparkExecutionContext sec, DataCharacteristics mcOut, ValueType[] schema) { //TODO
+ JavaPairRDD<LongWritable, Text> in = (JavaPairRDD<LongWritable, Text>) sec
+ .getRDDHandleForFrameObject(sec.getFrameObject(input1), FileFormat.LIBSVM);
+
+ //reblock libsvm to binary block
+ //return FrameRDDConverterUtils.libsvmToBinaryBlock(sec.getSparkContext(), in, mcOut, schema, _delim, _indexDelim);
+ return null;
+ }
+}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/ReblockSPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/ReblockSPInstruction.java
index a27a760..eaeb1de 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/spark/ReblockSPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/ReblockSPInstruction.java
@@ -40,6 +40,7 @@ import org.apache.sysds.runtime.instructions.spark.utils.FrameRDDConverterUtils;
import org.apache.sysds.runtime.instructions.spark.utils.RDDAggregateUtils;
import org.apache.sysds.runtime.instructions.spark.utils.RDDConverterUtils;
import org.apache.sysds.runtime.io.FileFormatPropertiesCSV;
+import org.apache.sysds.runtime.io.FileFormatPropertiesLIBSVM;
import org.apache.sysds.runtime.io.FileFormatPropertiesMM;
import org.apache.sysds.runtime.io.IOUtilFunctions;
import org.apache.sysds.runtime.matrix.data.FrameBlock;
@@ -66,16 +67,16 @@ public class ReblockSPInstruction extends UnarySPInstruction {
public static ReblockSPInstruction parseInstruction(String str) {
String parts[] = InstructionUtils.getInstructionPartsWithValueType(str);
String opcode = parts[0];
-
+
if(!opcode.equals("rblk")) {
throw new DMLRuntimeException("Incorrect opcode for ReblockSPInstruction:" + opcode);
}
-
+
CPOperand in = new CPOperand(parts[1]);
CPOperand out = new CPOperand(parts[2]);
int blen=Integer.parseInt(parts[3]);
boolean outputEmptyBlocks = Boolean.parseBoolean(parts[4]);
-
+
Operator op = null; // no operator for ReblockSPInstruction
return new ReblockSPInstruction(op, in, out, blen, blen, outputEmptyBlocks, opcode, str);
}
@@ -89,7 +90,7 @@ public class ReblockSPInstruction extends UnarySPInstruction {
DataCharacteristics mc = sec.getDataCharacteristics(input1.getName());
DataCharacteristics mcOut = sec.getDataCharacteristics(output.getName());
mcOut.set(mc.getRows(), mc.getCols(), blen, mc.getNonZeros());
-
+
//get the source format from the meta data
MetaDataFormat iimd = (MetaDataFormat) obj.getMetaData();
if(iimd == null)
@@ -102,11 +103,11 @@ public class ReblockSPInstruction extends UnarySPInstruction {
Statistics.decrementNoOfExecutedSPInst();
return;
}
-
+
//execute matrix/frame reblock
if( input1.getDataType() == DataType.MATRIX )
processMatrixReblockInstruction(sec, iimd.getFileFormat());
- else if( input1.getDataType() == DataType.FRAME )
+ else if(input1.getDataType() == DataType.FRAME)
processFrameReblockInstruction(sec, iimd.getFileFormat());
}
@@ -115,20 +116,20 @@ public class ReblockSPInstruction extends UnarySPInstruction {
MatrixObject mo = sec.getMatrixObject(input1.getName());
DataCharacteristics mc = sec.getDataCharacteristics(input1.getName());
DataCharacteristics mcOut = sec.getDataCharacteristics(output.getName());
-
+
if(fmt == FileFormat.TEXT || fmt == FileFormat.MM ) {
//get matrix market file properties if necessary
FileFormatPropertiesMM mmProps = (fmt == FileFormat.MM) ?
IOUtilFunctions.readAndParseMatrixMarketHeader(mo.getFileName()) : null;
-
+
//get the input textcell rdd
JavaPairRDD<LongWritable, Text> lines = (JavaPairRDD<LongWritable, Text>)
sec.getRDDHandleForMatrixObject(mo, fmt);
-
+
//convert textcell to binary block
JavaPairRDD<MatrixIndexes, MatrixBlock> out = RDDConverterUtils.textCellToBinaryBlock(
sec.getSparkContext(), lines, mcOut, outputEmptyBlocks, mmProps);
-
+
//put output RDD handle into symbol table
sec.setRDDHandleForVariable(output.getName(), out);
sec.addLineageRDD(output.getName(), input1.getName());
@@ -152,7 +153,7 @@ public class ReblockSPInstruction extends UnarySPInstruction {
fillValue = props.getFillValue();
naStrings = props.getNAStrings();
}
-
+
csvInstruction = new CSVReblockSPInstruction(null, input1, output, mcOut.getBlocksize(), mcOut.getBlocksize(), hasHeader, delim, fill, fillValue, "csvrblk", instString, naStrings);
csvInstruction.processInstruction(sec);
return;
@@ -169,20 +170,34 @@ public class ReblockSPInstruction extends UnarySPInstruction {
else if(fmt == FileFormat.BINARY) {
//BINARY BLOCK <- BINARY BLOCK (different sizes)
JavaPairRDD<MatrixIndexes, MatrixBlock> in1 = sec.getBinaryMatrixBlockRDDHandleForVariable(input1.getName());
-
+
boolean shuffleFreeReblock = mc.dimsKnown() && mcOut.dimsKnown()
&& (mc.getRows() < mcOut.getBlocksize() || mc.getBlocksize()%mcOut.getBlocksize() == 0)
&& (mc.getCols() < mcOut.getBlocksize() || mc.getBlocksize()%mcOut.getBlocksize() == 0);
-
+
JavaPairRDD<MatrixIndexes, MatrixBlock> out = in1
.flatMapToPair(new ExtractBlockForBinaryReblock(mc, mcOut));
if( !shuffleFreeReblock )
out = RDDAggregateUtils.mergeByKey(out, false);
-
+
//put output RDD handle into symbol table
sec.setRDDHandleForVariable(output.getName(), out);
sec.addLineageRDD(output.getName(), input1.getName());
}
+ else if(fmt == FileFormat.LIBSVM) {
+ String delim = IOUtilFunctions.LIBSVM_DELIM;
+ String indexDelim = IOUtilFunctions.LIBSVM_INDEX_DELIM;
+ if(mo.getFileFormatProperties() instanceof FileFormatPropertiesLIBSVM && mo
+ .getFileFormatProperties() != null) {
+ FileFormatPropertiesLIBSVM props = (FileFormatPropertiesLIBSVM) mo.getFileFormatProperties();
+ delim = props.getDelim();
+ indexDelim = props.getIndexDelim();
+ }
+
+ LIBSVMReblockSPInstruction libsvmInstruction = new LIBSVMReblockSPInstruction(null, input1, output,
+ mcOut.getBlocksize(), mcOut.getBlocksize(), "libsvmblk", delim, indexDelim, instString);
+ libsvmInstruction.processInstruction(sec);
+ }
else {
throw new DMLRuntimeException("The given format is not implemented "
+ "for ReblockSPInstruction:" + fmt.toString());
@@ -194,16 +209,16 @@ public class ReblockSPInstruction extends UnarySPInstruction {
{
FrameObject fo = sec.getFrameObject(input1.getName());
DataCharacteristics mcOut = sec.getDataCharacteristics(output.getName());
-
+
if(fmt == FileFormat.TEXT) {
//get the input textcell rdd
JavaPairRDD<LongWritable, Text> lines = (JavaPairRDD<LongWritable, Text>)
sec.getRDDHandleForFrameObject(fo, fmt);
-
+
//convert textcell to binary block
JavaPairRDD<Long, FrameBlock> out =
FrameRDDConverterUtils.textCellToBinaryBlock(sec.getSparkContext(), lines, mcOut, fo.getSchema());
-
+
//put output RDD handle into symbol table
sec.setRDDHandleForVariable(output.getName(), out);
sec.addLineageRDD(output.getName(), input1.getName());
@@ -227,10 +242,25 @@ public class ReblockSPInstruction extends UnarySPInstruction {
fillValue = props.getFillValue();
naStrings = props.getNAStrings();
}
-
+
csvInstruction = new CSVReblockSPInstruction(null, input1, output, mcOut.getBlocksize(), mcOut.getBlocksize(), hasHeader, delim, fill, fillValue, "csvrblk", instString, naStrings);
csvInstruction.processInstruction(sec);
}
+ else if(fmt == FileFormat.LIBSVM) {
+ String delim = IOUtilFunctions.LIBSVM_DELIM;
+ String indexDelim = IOUtilFunctions.LIBSVM_INDEX_DELIM;
+ if(fo.getFileFormatProperties() instanceof FileFormatPropertiesLIBSVM && fo
+ .getFileFormatProperties() != null) {
+ FileFormatPropertiesLIBSVM props = (FileFormatPropertiesLIBSVM) fo.getFileFormatProperties();
+ delim = props.getDelim();
+ indexDelim = props.getIndexDelim();
+ }
+ LIBSVMReblockSPInstruction libsvmInstruction = new LIBSVMReblockSPInstruction(null, input1, output,
+ mcOut.getBlocksize(), mcOut.getBlocksize(), "libsvmblk", delim, indexDelim, instString);
+ libsvmInstruction.processInstruction(sec);
+
+ }
+
else {
throw new DMLRuntimeException("The given format is not implemented "
+ "for ReblockSPInstruction: " + fmt.toString());
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 297bb5b..c9935b1 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
@@ -8,7 +8,7 @@
* 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
@@ -29,14 +29,14 @@ import org.apache.sysds.utils.Statistics;
public abstract class SPInstruction extends Instruction {
- public enum SPType {
- MAPMM, MAPMMCHAIN, CPMM, RMM, TSMM, TSMM2, PMM, ZIPMM, PMAPMM, //matrix multiplication instructions
+ public enum SPType {
+ MAPMM, MAPMMCHAIN, CPMM, RMM, TSMM, TSMM2, PMM, ZIPMM, PMAPMM, //matrix multiplication instructions
MatrixIndexing, Reorg, Binary, Ternary,
- AggregateUnary, AggregateTernary, Reblock, CSVReblock,
+ AggregateUnary, AggregateTernary, Reblock, CSVReblock, LIBSVMReblock,
Builtin, Unary, BuiltinNary, MultiReturnBuiltin, Checkpoint, Compression, DeCompression, Cast,
- CentralMoment, Covariance, QSort, QPick,
- ParameterizedBuiltin, MAppend, RAppend, GAppend, GAlignedAppend, Rand,
- MatrixReshape, Ctable, Quaternary, CumsumAggregate, CumsumOffset, BinUaggChain, UaggOuterChain,
+ CentralMoment, Covariance, QSort, QPick,
+ ParameterizedBuiltin, MAppend, RAppend, GAppend, GAlignedAppend, Rand,
+ MatrixReshape, Ctable, Quaternary, CumsumAggregate, CumsumOffset, BinUaggChain, UaggOuterChain,
Write, SpoofFused, Dnn
}
@@ -56,7 +56,7 @@ public abstract class SPInstruction extends Instruction {
instOpcode = opcode;
_requiresLabelUpdate = super.requiresLabelUpdate();
}
-
+
@Override
public IType getType() {
return IType.SPARK;
@@ -65,7 +65,7 @@ public abstract class SPInstruction extends Instruction {
public SPType getSPInstructionType() {
return _sptype;
}
-
+
@Override
public boolean requiresLabelUpdate() {
return _requiresLabelUpdate;
@@ -75,12 +75,12 @@ public abstract class SPInstruction extends Instruction {
public String getGraphString() {
return getOpcode();
}
-
+
@Override
public Instruction preprocessInstruction(ExecutionContext ec) {
//default pre-process behavior (e.g., debug state)
Instruction tmp = super.preprocessInstruction(ec);
-
+
//instruction patching
if( tmp.requiresLabelUpdate() ) //update labels only if required
{
@@ -88,21 +88,21 @@ public abstract class SPInstruction extends Instruction {
String updInst = CPInstruction.updateLabels(tmp.toString(), ec.getVariables());
tmp = SPInstructionParser.parseSingleInstruction(updInst);
}
-
+
//robustness federated instructions (runtime assignment)
tmp = FEDInstructionUtils.checkAndReplaceSP(tmp, ec);
-
+
return tmp;
}
- @Override
+ @Override
public abstract void processInstruction(ExecutionContext ec);
@Override
public void postprocessInstruction(ExecutionContext ec) {
//maintain statistics
Statistics.incrementNoOfExecutedSPInst();
-
+
//default post-process behavior
super.postprocessInstruction(ec);
}
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/WriteSPInstruction.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/WriteSPInstruction.java
index 3f71d5c..d9b21d3 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/spark/WriteSPInstruction.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/WriteSPInstruction.java
@@ -40,6 +40,7 @@ import org.apache.sysds.runtime.instructions.spark.utils.FrameRDDConverterUtils.
import org.apache.sysds.runtime.instructions.spark.utils.RDDConverterUtils;
import org.apache.sysds.runtime.io.FileFormatProperties;
import org.apache.sysds.runtime.io.FileFormatPropertiesCSV;
+import org.apache.sysds.runtime.io.FileFormatPropertiesLIBSVM;
import org.apache.sysds.runtime.lineage.LineageItem;
import org.apache.sysds.runtime.lineage.LineageItemUtils;
import org.apache.sysds.runtime.lineage.LineageTraceable;
@@ -71,24 +72,25 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
public static WriteSPInstruction parseInstruction ( String str ) {
String[] parts = InstructionUtils.getInstructionPartsWithValueType ( str );
String opcode = parts[0];
-
+
if( !opcode.equals("write") ) {
throw new DMLRuntimeException("Unsupported opcode");
}
-
- // All write instructions have 3 parameters, except in case of delimited/csv file.
+
+ // All write instructions have 3 parameters, except in case of delimited/csv/libsvm file.
// Write instructions for csv files also include three additional parameters (hasHeader, delimiter, sparse)
+ // Write instructions for libsvm files also include three additional parameters (delimiter, index delimiter, sparse)
if ( parts.length != 5 && parts.length != 9 ) {
throw new DMLRuntimeException("Invalid number of operands in write instruction: " + str);
}
-
+
// _mVar2·MATRIX·DOUBLE
CPOperand in1 = new CPOperand(parts[1]);
CPOperand in2 = new CPOperand(parts[2]);
CPOperand in3 = new CPOperand(parts[3]);
-
- WriteSPInstruction inst = new WriteSPInstruction(in1, in2, in3, opcode, str);
-
+
+ WriteSPInstruction inst = new WriteSPInstruction(in1, in2, in3, opcode, str);
+
if ( in3.getName().equalsIgnoreCase("csv") ) {
boolean hasHeader = Boolean.parseBoolean(parts[4]);
String delim = parts[5];
@@ -97,7 +99,17 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
inst.setFormatProperties(formatProperties);
CPOperand in4 = new CPOperand(parts[8]);
inst.input4 = in4;
- } else {
+ }
+ else if(in3.getName().equalsIgnoreCase("libsvm")) {
+ String delim = parts[4];
+ String indexDelim = parts[5];
+ boolean sparse = Boolean.parseBoolean(parts[6]);
+ FileFormatProperties formatProperties = new FileFormatPropertiesLIBSVM(delim, indexDelim, sparse);
+ inst.setFormatProperties(formatProperties);
+ CPOperand in4 = new CPOperand(parts[8]);
+ inst.input4 = in4;
+ }
+ else {
FileFormatProperties ffp = new FileFormatProperties();
CPOperand in4 = new CPOperand(parts[4]);
@@ -106,24 +118,24 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
}
return inst;
}
-
-
+
+
public FileFormatProperties getFormatProperties() {
return formatProperties;
}
-
+
public void setFormatProperties(FileFormatProperties prop) {
formatProperties = prop;
}
-
- public CPOperand getInput1() {
+
+ public CPOperand getInput1() {
return input1;
}
-
+
public CPOperand getInput2() {
return input2;
}
-
+
@Override
public void processInstruction(ExecutionContext ec) {
SparkExecutionContext sec = (SparkExecutionContext) ec;
@@ -133,9 +145,9 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
String desc = ec.getScalarInput(input4.getName(), ValueType.STRING, input4.isLiteral()).getStringValue();
formatProperties.setDescription(desc);
- ValueType[] schema = (input1.getDataType()==DataType.FRAME) ?
+ ValueType[] schema = (input1.getDataType()==DataType.FRAME) ?
sec.getFrameObject(input1.getName()).getSchema() : null;
-
+
try
{
//if the file already exists on HDFS, remove it.
@@ -143,7 +155,7 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
//prepare output info according to meta data
FileFormat fmt = FileFormat.safeValueOf(input3.getName());
-
+
//core matrix/frame write
switch( input1.getDataType() ) {
case MATRIX: processMatrixWriteInstruction(sec, fname, fmt); break;
@@ -158,13 +170,13 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
}
}
- protected void processMatrixWriteInstruction(SparkExecutionContext sec, String fname, FileFormat fmt)
+ protected void processMatrixWriteInstruction(SparkExecutionContext sec, String fname, FileFormat fmt)
throws IOException
{
//get input rdd
JavaPairRDD<MatrixIndexes,MatrixBlock> in1 = sec.getBinaryMatrixBlockRDDHandleForVariable( input1.getName() );
DataCharacteristics mc = sec.getDataCharacteristics(input1.getName());
-
+
if( fmt == FileFormat.MM || fmt == FileFormat.TEXT )
{
//piggyback nnz maintenance on write
@@ -173,7 +185,7 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
-
+
JavaRDD<String> header = null;
if( fmt == FileFormat.MM ) {
ArrayList<String> headerContainer = new ArrayList<>(1);
@@ -184,13 +196,13 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
headerContainer.add(headerStr);
header = sec.getSparkContext().parallelize(headerContainer);
}
-
+
JavaRDD<String> ijv = RDDConverterUtils.binaryBlockToTextCell(in1, mc);
if(header != null)
customSaveTextFile(header.union(ijv), fname, true);
else
customSaveTextFile(ijv, fname, false);
-
+
if( !mc.nnzKnown() )
mc.setNonZeros( aNnz.value() );
}
@@ -200,20 +212,20 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
throw new IOException("Write of matrices with zero rows or columns"
+ " not supported ("+mc.getRows()+"x"+mc.getCols()+").");
}
-
+
LongAccumulator aNnz = null;
-
+
//piggyback nnz computation on actual write
if( !mc.nnzKnown() ) {
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
-
+
JavaRDD<String> out = RDDConverterUtils.binaryBlockToCsv(
in1, mc, (FileFormatPropertiesCSV) formatProperties, true);
customSaveTextFile(out, fname, false);
-
+
if( !mc.nnzKnown() )
mc.setNonZeros(aNnz.value().longValue());
}
@@ -224,10 +236,33 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
}
-
+
//save binary block rdd on hdfs
in1.saveAsHadoopFile(fname, MatrixIndexes.class, MatrixBlock.class, SequenceFileOutputFormat.class);
-
+
+ if(!mc.nnzKnown())
+ mc.setNonZeros(aNnz.value().longValue());
+ }
+ else if(fmt == FileFormat.LIBSVM) {
+ if(mc.getRows() == 0 || mc.getCols() == 0) {
+ throw new IOException(
+ "Write of matrices with zero rows or columns" + " not supported (" + mc.getRows() + "x" + mc
+ .getCols() + ").");
+ }
+
+ LongAccumulator aNnz = null;
+
+ //piggyback nnz computation on actual write
+ if(!mc.nnzKnown()) {
+ aNnz = sec.getSparkContext().sc().longAccumulator("nnz");
+ in1 = in1.mapValues(new ComputeBinaryBlockNnzFunction(aNnz));
+ }
+
+ JavaRDD<String> out = RDDConverterUtils.binaryBlockToLibsvm(in1,
+ mc, (FileFormatPropertiesLIBSVM) formatProperties, true);
+
+ customSaveTextFile(out, fname, false);
+
if( !mc.nnzKnown() )
mc.setNonZeros(aNnz.value().longValue());
}
@@ -235,19 +270,19 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
//unsupported formats: binarycell (not externalized)
throw new DMLRuntimeException("Unexpected data format: " + fmt.toString());
}
-
+
// write meta data file
HDFSTool.writeMetaDataFile (fname + ".mtd", ValueType.FP64, mc, fmt, formatProperties);
}
- protected void processFrameWriteInstruction(SparkExecutionContext sec, String fname, FileFormat fmt, ValueType[] schema)
+ protected void processFrameWriteInstruction(SparkExecutionContext sec, String fname, FileFormat fmt, ValueType[] schema)
throws IOException
{
//get input rdd
JavaPairRDD<Long,FrameBlock> in1 = sec
.getFrameBinaryBlockRDDHandleForVariable(input1.getName());
DataCharacteristics mc = sec.getDataCharacteristics(input1.getName());
-
+
switch(fmt) {
case TEXT: {
JavaRDD<String> out = FrameRDDConverterUtils.binaryBlockToTextCell(in1, mc);
@@ -260,6 +295,13 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
customSaveTextFile(out, fname, false);
break;
}
+ case LIBSVM: {
+ // TODO: implement for libsvm
+ // FileFormatPropertiesCSV props = (formatProperties!=null) ?(FileFormatPropertiesCSV) formatProperties : null;
+ // JavaRDD<String> out = FrameRDDConverterUtils.binaryBlockToCsv(in1, mc, props, true);
+ // customSaveTextFile(out, fname, false);
+ break;
+ }
case BINARY: {
JavaPairRDD<LongWritable,FrameBlock> out = in1.mapToPair(new LongFrameToLongWritableFrameFunction());
out.saveAsHadoopFile(fname, LongWritable.class, FrameBlock.class, SequenceFileOutputFormat.class);
@@ -268,7 +310,7 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
default:
throw new DMLRuntimeException("Unexpected data format: " + fmt.toString());
}
-
+
// write meta data file
HDFSTool.writeMetaDataFile(fname + ".mtd", input1.getValueType(), schema, DataType.FRAME, mc, fmt, formatProperties);
}
@@ -281,10 +323,10 @@ public class WriteSPInstruction extends SPInstruction implements LineageTraceabl
while(HDFSTool.existsFileOnHDFS(randFName)) {
randFName = fname + "_" + rand.nextLong() + "_" + rand.nextLong();
}
-
+
rdd.saveAsTextFile(randFName);
HDFSTool.mergeIntoSingleFile(randFName, fname); // Faster version :)
-
+
// rdd.coalesce(1, true).saveAsTextFile(randFName);
// MapReduceTool.copyFileOnHDFS(randFName + "/part-00000", fname);
} catch (IOException e) {
diff --git a/src/main/java/org/apache/sysds/runtime/instructions/spark/utils/RDDConverterUtils.java b/src/main/java/org/apache/sysds/runtime/instructions/spark/utils/RDDConverterUtils.java
index 4fc175d..7b28a0b 100644
--- a/src/main/java/org/apache/sysds/runtime/instructions/spark/utils/RDDConverterUtils.java
+++ b/src/main/java/org/apache/sysds/runtime/instructions/spark/utils/RDDConverterUtils.java
@@ -32,10 +32,7 @@ import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.FlatMapFunction;
-import org.apache.spark.api.java.function.Function;
-import org.apache.spark.api.java.function.PairFlatMapFunction;
-import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.api.java.function.*;
import org.apache.spark.ml.feature.LabeledPoint;
import org.apache.spark.ml.linalg.DenseVector;
import org.apache.spark.ml.linalg.SparseVector;
@@ -57,11 +54,13 @@ import org.apache.sysds.common.Types.ValueType;
import org.apache.sysds.conf.ConfigurationManager;
import org.apache.sysds.hops.OptimizerUtils;
import org.apache.sysds.runtime.DMLRuntimeException;
+import org.apache.sysds.runtime.data.SparseBlock;
import org.apache.sysds.runtime.instructions.spark.data.ReblockBuffer;
import org.apache.sysds.runtime.instructions.spark.data.SerLongWritable;
import org.apache.sysds.runtime.instructions.spark.data.SerText;
import org.apache.sysds.runtime.instructions.spark.functions.ConvertMatrixBlockToIJVLines;
import org.apache.sysds.runtime.io.FileFormatPropertiesCSV;
+import org.apache.sysds.runtime.io.FileFormatPropertiesLIBSVM;
import org.apache.sysds.runtime.io.FileFormatPropertiesMM;
import org.apache.sysds.runtime.io.IOUtilFunctions;
import org.apache.sysds.runtime.matrix.data.MatrixBlock;
@@ -86,15 +85,15 @@ public class RDDConverterUtils {
JavaPairRDD<MatrixIndexes, MatrixBlock> out = input.values()
.mapPartitionsToPair(new TextToBinaryBlockFunction(mcOut, mmProps));
- //inject empty blocks (if necessary)
+ //inject empty blocks (if necessary)
if( outputEmptyBlocks && mcOut.mightHaveEmptyBlocks() ) {
- out = out.union(
+ out = out.union(
SparkUtils.getEmptyBlockRDD(sc, mcOut) );
}
-
+
//aggregate partial matrix blocks
- out = RDDAggregateUtils.mergeByKey(out, false);
-
+ out = RDDAggregateUtils.mergeByKey(out, false);
+
return out;
}
@@ -105,32 +104,32 @@ public class RDDConverterUtils {
JavaPairRDD<MatrixIndexes, MatrixBlock> out = input
.mapPartitionsToPair(new BinaryCellToBinaryBlockFunction(mcOut));
- //inject empty blocks (if necessary)
+ //inject empty blocks (if necessary)
if( outputEmptyBlocks && mcOut.mightHaveEmptyBlocks() ) {
- out = out.union(
+ out = out.union(
SparkUtils.getEmptyBlockRDD(sc, mcOut) );
}
-
+
//aggregate partial matrix blocks
- out = RDDAggregateUtils.mergeByKey(out, false);
-
+ out = RDDAggregateUtils.mergeByKey(out, false);
+
return out;
}
/**
- * Converter from binary block rdd to rdd of labeled points. Note that the input needs to be
+ * Converter from binary block rdd to rdd of labeled points. Note that the input needs to be
* reblocked to satisfy the 'clen <= blen' constraint.
- *
+ *
* @param in matrix as {@code JavaPairRDD<MatrixIndexes, MatrixBlock>}
* @return JavaRDD of labeled points
*/
- public static JavaRDD<LabeledPoint> binaryBlockToLabeledPoints(JavaPairRDD<MatrixIndexes, MatrixBlock> in)
+ public static JavaRDD<LabeledPoint> binaryBlockToLabeledPoints(JavaPairRDD<MatrixIndexes, MatrixBlock> in)
{
//convert indexed binary block input to collection of labeled points
JavaRDD<LabeledPoint> pointrdd = in
.values()
.flatMap(new PrepareBinaryBlockFunction());
-
+
return pointrdd;
}
@@ -141,32 +140,53 @@ public class RDDConverterUtils {
public static JavaRDD<String> binaryBlockToCsv(JavaPairRDD<MatrixIndexes,MatrixBlock> in, DataCharacteristics mcIn, FileFormatPropertiesCSV props, boolean strict)
{
JavaPairRDD<MatrixIndexes,MatrixBlock> input = in;
-
+
//fast path without, general case with shuffle
if( mcIn.getCols()>mcIn.getBlocksize() ) {
//create row partitioned matrix
input = input
.flatMapToPair(new SliceBinaryBlockToRowsFunction(mcIn.getBlocksize()))
.groupByKey()
- .mapToPair(new ConcatenateBlocksFunction(mcIn.getCols(), mcIn.getBlocksize()));
+ .mapToPair(new ConcatenateBlocksFunction(mcIn.getCols(), mcIn.getBlocksize()));
}
-
+
//sort if required (on blocks/rows)
if( strict ) {
input = input.sortByKey(true);
}
-
+
//convert binary block to csv (from blocks/rows)
- JavaRDD<String> out = input
- .flatMap(new BinaryBlockToCSVFunction(props));
-
+ JavaRDD<String> out = input.flatMap(new BinaryBlockToCSVFunction(props));
+
+ return out;
+ }
+
+ public static JavaRDD<String> binaryBlockToLibsvm(JavaPairRDD<MatrixIndexes, MatrixBlock> in,
+ DataCharacteristics mcIn, FileFormatPropertiesLIBSVM props, boolean strict) {
+ JavaPairRDD<MatrixIndexes, MatrixBlock> input = in;
+
+ //fast path without, general case with shuffle
+ if(mcIn.getCols() > mcIn.getBlocksize()) {
+ //create row partitioned matrix
+ input = input.flatMapToPair(new SliceBinaryBlockToRowsFunction(mcIn.getBlocksize())).groupByKey()
+ .mapToPair(new ConcatenateBlocksFunction(mcIn.getCols(), mcIn.getBlocksize()));
+ }
+
+ //sort if required (on blocks/rows)
+ if(strict) {
+ input = input.sortByKey(true);
+ }
+
+ //convert binary block to libsvm (from blocks/rows)
+ JavaRDD<String> out = input.flatMap(new BinaryBlockToLIBSVMFunction(props));
+
return out;
}
public static JavaPairRDD<MatrixIndexes, MatrixBlock> csvToBinaryBlock(JavaSparkContext sc,
JavaPairRDD<LongWritable, Text> input, DataCharacteristics mc,
boolean hasHeader, String delim, boolean fill, double fillValue, Set<String> naStrings) {
-
+
//determine unknown dimensions and sparsity if required
//(w/ robustness for mistakenly counted header in nnz)
if( !mc.dimsKnown(true) ) {
@@ -178,32 +198,32 @@ public class RDDConverterUtils {
long nnz = Math.min(rlen*clen, UtilFunctions.toLong(aNnz.value()));
mc.set(rlen, clen, mc.getBlocksize(), nnz);
}
-
+
//prepare csv w/ row indexes (sorted by filenames)
JavaPairRDD<Text,Long> prepinput = input.values()
.zipWithIndex(); //zip row index
-
+
//convert csv rdd to binary block rdd (w/ partial blocks)
boolean sparse = requiresSparseAllocation(prepinput, mc);
- JavaPairRDD<MatrixIndexes, MatrixBlock> out =
+ JavaPairRDD<MatrixIndexes, MatrixBlock> out =
prepinput.mapPartitionsToPair(new CSVToBinaryBlockFunction(
mc, sparse, hasHeader, delim, fill, fillValue, naStrings));
-
- //aggregate partial matrix blocks (w/ preferred number of output
+
+ //aggregate partial matrix blocks (w/ preferred number of output
//partitions as the data is likely smaller in binary block format,
//but also to bound the size of partitions for compressed inputs)
int parts = SparkUtils.getNumPreferredPartitions(mc, out);
- return RDDAggregateUtils.mergeByKey(out, parts, false);
+ return RDDAggregateUtils.mergeByKey(out, parts, false);
}
-
+
public static JavaPairRDD<MatrixIndexes, MatrixBlock> csvToBinaryBlock(JavaSparkContext sc,
JavaRDD<String> input, DataCharacteristics mcOut,
- boolean hasHeader, String delim, boolean fill, double fillValue, Set<String> naStrings)
+ boolean hasHeader, String delim, boolean fill, double fillValue, Set<String> naStrings)
{
//convert string rdd to serializable longwritable/text
JavaPairRDD<LongWritable, Text> prepinput =
input.mapToPair(new StringToSerTextFunction());
-
+
//convert to binary block
return csvToBinaryBlock(sc, prepinput, mcOut, hasHeader, delim, fill, fillValue, naStrings);
}
@@ -216,33 +236,33 @@ public class RDDConverterUtils {
LongAccumulator aNnz = sc.sc().longAccumulator("nnz");
JavaRDD<Row> tmp = df.javaRDD().map(new DataFrameAnalysisFunction(aNnz, containsID, isVector));
long rlen = tmp.count();
- long clen = !isVector ? df.columns().length - (containsID?1:0) :
+ long clen = !isVector ? df.columns().length - (containsID?1:0) :
((Vector) tmp.first().get(containsID?1:0)).size();
long nnz = UtilFunctions.toLong(aNnz.value());
mc.set(rlen, clen, mc.getBlocksize(), nnz);
}
-
+
//ensure valid blocksizes
if( mc.getBlocksize()<=1 )
mc.setBlocksize(ConfigurationManager.getBlocksize());
-
+
//construct or reuse row ids
JavaPairRDD<Row, Long> prepinput = containsID ?
df.javaRDD().mapToPair(new DataFrameExtractIDFunction(
df.schema().fieldIndex(DF_ID_COLUMN))) :
df.javaRDD().zipWithIndex(); //zip row index
-
+
//convert csv rdd to binary block rdd (w/ partial blocks)
boolean sparse = requiresSparseAllocation(prepinput, mc);
- JavaPairRDD<MatrixIndexes, MatrixBlock> out =
+ JavaPairRDD<MatrixIndexes, MatrixBlock> out =
prepinput.mapPartitionsToPair(
new DataFrameToBinaryBlockFunction(mc, sparse, containsID, isVector));
-
- //aggregate partial matrix blocks (w/ preferred number of output
+
+ //aggregate partial matrix blocks (w/ preferred number of output
//partitions as the data is likely smaller in binary block format,
//but also to bound the size of partitions for compressed inputs)
int parts = SparkUtils.getNumPreferredPartitions(mc, out);
- return RDDAggregateUtils.mergeByKey(out, parts, false);
+ return RDDAggregateUtils.mergeByKey(out, parts, false);
}
public static Dataset<Row> binaryBlockToDataFrame(SparkSession sparkSession,
@@ -250,12 +270,12 @@ public class RDDConverterUtils {
{
if( !mc.colsKnown() )
throw new RuntimeException("Number of columns needed to convert binary block to data frame.");
-
+
//slice blocks into rows, align and convert into data frame rows
JavaRDD<Row> rowsRDD = in
.flatMapToPair(new SliceBinaryBlockToRowsFunction(mc.getBlocksize()))
.groupByKey().map(new ConvertRowBlocksToRows((int)mc.getCols(), mc.getBlocksize(), toVector));
-
+
//create data frame schema
List<StructField> fields = new ArrayList<>();
fields.add(DataTypes.createStructField(DF_ID_COLUMN, DataTypes.DoubleType, false));
@@ -265,7 +285,7 @@ public class RDDConverterUtils {
for(int i = 1; i <= mc.getCols(); i++)
fields.add(DataTypes.createStructField("C"+i, DataTypes.DoubleType, false));
}
-
+
//rdd to data frame conversion
return sparkSession.createDataFrame(rowsRDD.rdd(), DataTypes.createStructType(fields));
}
@@ -278,21 +298,21 @@ public class RDDConverterUtils {
}
/**
- * Converts a libsvm text input file into two binary block matrices for features
- * and labels, and saves these to the specified output files. This call also deletes
- * existing files at the specified output locations, as well as determines and
- * writes the meta data files of both output matrices.
+ * Converts a libsvm text input file into two binary block matrices for features
+ * and labels, and saves these to the specified output files. This call also deletes
+ * existing files at the specified output locations, as well as determines and
+ * writes the meta data files of both output matrices.
* <p>
- * Note: We use {@code org.apache.spark.mllib.util.MLUtils.loadLibSVMFile} for parsing
+ * Note: We use {@code org.apache.spark.mllib.util.MLUtils.loadLibSVMFile} for parsing
* the libsvm input files in order to ensure consistency with Spark.
- *
+ *
* @param sc java spark context
* @param pathIn path to libsvm input file
* @param pathX path to binary block output file of features
* @param pathY path to binary block output file of labels
* @param mcOutX matrix characteristics of output matrix X
*/
- public static void libsvmToBinaryBlock(JavaSparkContext sc, String pathIn,
+ public static void libsvmToBinaryBlock(JavaSparkContext sc, String pathIn,
String pathX, String pathY, DataCharacteristics mcOutX)
{
if( !mcOutX.dimsKnown() )
@@ -302,17 +322,17 @@ public class RDDConverterUtils {
//cleanup existing output files
HDFSTool.deleteFileIfExistOnHDFS(pathX);
HDFSTool.deleteFileIfExistOnHDFS(pathY);
-
+
//convert libsvm to labeled points
int numFeatures = (int) mcOutX.getCols();
int numPartitions = SparkUtils.getNumPreferredPartitions(mcOutX, null);
- JavaRDD<org.apache.spark.mllib.regression.LabeledPoint> lpoints =
+ JavaRDD<org.apache.spark.mllib.regression.LabeledPoint> lpoints =
MLUtils.loadLibSVMFile(sc.sc(), pathIn, numFeatures, numPartitions).toJavaRDD();
-
+
//append row index and best-effort caching to avoid repeated text parsing
- JavaPairRDD<org.apache.spark.mllib.regression.LabeledPoint,Long> ilpoints =
- lpoints.zipWithIndex().persist(StorageLevel.MEMORY_AND_DISK());
-
+ JavaPairRDD<org.apache.spark.mllib.regression.LabeledPoint,Long> ilpoints =
+ lpoints.zipWithIndex().persist(StorageLevel.MEMORY_AND_DISK());
+
//extract labels and convert to binary block
DataCharacteristics mc1 = new MatrixCharacteristics(mcOutX.getRows(), 1, mcOutX.getBlocksize(), -1);
LongAccumulator aNnz1 = sc.sc().longAccumulator("nnz");
@@ -323,7 +343,7 @@ public class RDDConverterUtils {
out1.saveAsHadoopFile(pathY, MatrixIndexes.class, MatrixBlock.class, SequenceFileOutputFormat.class);
mc1.setNonZeros(aNnz1.value()); //update nnz after triggered save
HDFSTool.writeMetaDataFile(pathY+".mtd", ValueType.FP64, mc1, FileFormat.BINARY);
-
+
//extract data and convert to binary block
DataCharacteristics mc2 = new MatrixCharacteristics(mcOutX.getRows(), mcOutX.getCols(), mcOutX.getBlocksize(), -1);
LongAccumulator aNnz2 = sc.sc().longAccumulator("nnz");
@@ -333,7 +353,7 @@ public class RDDConverterUtils {
out2.saveAsHadoopFile(pathX, MatrixIndexes.class, MatrixBlock.class, SequenceFileOutputFormat.class);
mc2.setNonZeros(aNnz2.value()); //update nnz after triggered save
HDFSTool.writeMetaDataFile(pathX+".mtd", ValueType.FP64, mc2, FileFormat.BINARY);
-
+
//asynchronous cleanup of cached intermediates
ilpoints.unpersist(false);
}
@@ -341,7 +361,7 @@ public class RDDConverterUtils {
throw new DMLRuntimeException(ex);
}
}
-
+
public static JavaPairRDD<LongWritable, Text> stringToSerializableText(JavaPairRDD<Long,String> in)
{
return in.mapToPair(new TextToSerTextFunction());
@@ -353,7 +373,7 @@ public class RDDConverterUtils {
mc.getRows(), mc.getCols(), mc.getNonZeros())) ) {
return true;
}
-
+
//if dense evaluate expected rows per partition to handle wide matrices
//(pick sparse representation if fraction of rows per block less than sparse theshold)
double datasize = OptimizerUtils.estimatePartitionedSizeExactSparsity(mc);
@@ -367,17 +387,17 @@ public class RDDConverterUtils {
private static int countNnz(Object vect, boolean isVector, int off) {
if( isVector ) //note: numNonzeros scans entries but handles sparse/dense
return ((Vector) vect).numNonzeros();
- else
+ else
return countNnz(vect, isVector, off, ((Row)vect).length());
}
/**
* Count the number of non-zeros for a subrange of the given row.
- *
+ *
* @param vect row object (row of basic types or row including a vector)
* @param isVector if the row includes a vector
- * @param pos physical position
- * @param cu logical upper column index (exclusive)
+ * @param pos physical position
+ * @param cu logical upper column index (exclusive)
* @return number of non-zeros.
*/
private static int countNnz(Object vect, boolean isVector, int pos, int cu ) {
@@ -404,69 +424,69 @@ public class RDDConverterUtils {
}
return lnnz;
}
-
+
private static Vector createVector(MatrixBlock row) {
if( row.isEmptyBlock(false) ) //EMPTY SPARSE ROW
return Vectors.sparse(row.getNumColumns(), new int[0], new double[0]);
else if( row.isInSparseFormat() ) //SPARSE ROW
- return Vectors.sparse(row.getNumColumns(),
+ return Vectors.sparse(row.getNumColumns(),
row.getSparseBlock().indexes(0), row.getSparseBlock().values(0));
else // DENSE ROW
return Vectors.dense(row.getDenseBlockValues());
}
-
+
/////////////////////////////////
// BINARYBLOCK-SPECIFIC FUNCTIONS
/**
* This function converts a binary block input (<X,y>) into mllib's labeled points. Note that
* this function requires prior reblocking if the number of columns is larger than the column
- * block size.
+ * block size.
*/
- private static class PrepareBinaryBlockFunction implements FlatMapFunction<MatrixBlock, LabeledPoint>
+ private static class PrepareBinaryBlockFunction implements FlatMapFunction<MatrixBlock, LabeledPoint>
{
private static final long serialVersionUID = -6590259914203201585L;
@Override
- public Iterator<LabeledPoint> call(MatrixBlock arg0)
- throws Exception
+ public Iterator<LabeledPoint> call(MatrixBlock arg0)
+ throws Exception
{
ArrayList<LabeledPoint> ret = new ArrayList<>();
for( int i=0; i<arg0.getNumRows(); i++ ) {
MatrixBlock tmp = arg0.slice(i, i, 0, arg0.getNumColumns()-2, new MatrixBlock());
ret.add(new LabeledPoint(arg0.getValue(i, arg0.getNumColumns()-1), createVector(tmp)));
}
-
+
return ret.iterator();
}
}
-
+
/////////////////////////////////
// TEXTCELL-SPECIFIC FUNCTIONS
-
+
private static abstract class CellToBinaryBlockFunction implements Serializable
{
private static final long serialVersionUID = 4205331295408335933L;
-
+
//internal buffer size (aligned w/ default matrix block size)
protected static final int BUFFER_SIZE = 4 * 1000 * 1000; //4M elements (32MB)
protected int _bufflen = -1;
-
+
protected long _rlen = -1;
protected long _clen = -1;
protected int _blen = -1;
-
+
protected CellToBinaryBlockFunction(DataCharacteristics mc)
{
_rlen = mc.getRows();
_clen = mc.getCols();
_blen = mc.getBlocksize();
-
+
//determine upper bounded buffer len
_bufflen = (int) Math.min(_rlen*_clen, BUFFER_SIZE);
}
- protected void flushBufferToList( ReblockBuffer rbuff, ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret )
+ protected void flushBufferToList( ReblockBuffer rbuff, ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret )
throws DMLRuntimeException
{
rbuff.flushBufferToBinaryBlocks().stream() // prevent library dependencies
@@ -474,26 +494,26 @@ public class RDDConverterUtils {
}
}
- private static class TextToBinaryBlockFunction extends CellToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Text>,MatrixIndexes,MatrixBlock>
+ private static class TextToBinaryBlockFunction extends CellToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Text>,MatrixIndexes,MatrixBlock>
{
private static final long serialVersionUID = 4907483236186747224L;
private final FileFormatPropertiesMM _mmProps;
-
+
protected TextToBinaryBlockFunction(DataCharacteristics mc, FileFormatPropertiesMM mmProps) {
super(mc);
_mmProps = mmProps;
}
@Override
- public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Text> arg0)
- throws Exception
+ public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Text> arg0)
+ throws Exception
{
ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret = new ArrayList<>();
ReblockBuffer rbuff = new ReblockBuffer(_bufflen, _rlen, _clen, _blen);
FastStringTokenizer st = new FastStringTokenizer(' ');
boolean first = false;
-
+
while( arg0.hasNext() ) {
//get input string (ignore matrix market comments as well as
//first row which indicates meta data, i.e., <nrow> <ncol> <nnz>)
@@ -506,63 +526,63 @@ public class RDDConverterUtils {
first = false;
continue;
}
-
+
//parse input ijv triple
st.reset( strVal.toString() ); //reinit tokenizer
long row = st.nextLong();
long col = st.nextLong();
- double val = (_mmProps == null) ? st.nextDouble() :
+ double val = (_mmProps == null) ? st.nextDouble() :
_mmProps.isPatternField() ? 1 : _mmProps.isIntField() ? st.nextLong() : st.nextDouble();
-
+
//flush buffer if necessary
if( rbuff.getSize() >= rbuff.getCapacity() )
flushBufferToList(rbuff, ret);
-
+
//add value to reblock buffer
rbuff.appendCell(row, col, val);
if( _mmProps != null && _mmProps.isSymmetric() && row!=col )
rbuff.appendCell(col, row, val);
}
-
+
//final flush buffer
flushBufferToList(rbuff, ret);
-
+
return ret.iterator();
}
}
- private static class TextToSerTextFunction implements PairFunction<Tuple2<Long,String>,LongWritable,Text>
+ private static class TextToSerTextFunction implements PairFunction<Tuple2<Long,String>,LongWritable,Text>
{
private static final long serialVersionUID = 2286037080400222528L;
@Override
- public Tuple2<LongWritable, Text> call(Tuple2<Long, String> arg0)
- throws Exception
+ public Tuple2<LongWritable, Text> call(Tuple2<Long, String> arg0)
+ throws Exception
{
SerLongWritable slarg = new SerLongWritable(arg0._1());
- SerText starg = new SerText(arg0._2());
+ SerText starg = new SerText(arg0._2());
return new Tuple2<>(slarg, starg);
}
}
- private static class StringToSerTextFunction implements PairFunction<String, LongWritable, Text>
+ private static class StringToSerTextFunction implements PairFunction<String, LongWritable, Text>
{
private static final long serialVersionUID = 2286037080400222528L;
@Override
- public Tuple2<LongWritable, Text> call(String arg0)
- throws Exception
+ public Tuple2<LongWritable, Text> call(String arg0)
+ throws Exception
{
SerLongWritable slarg = new SerLongWritable(1L);
SerText starg = new SerText(arg0);
return new Tuple2<>(slarg, starg);
}
}
-
+
/////////////////////////////////
// BINARYCELL-SPECIFIC FUNCTIONS
- public static class BinaryCellToBinaryBlockFunction extends CellToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<MatrixIndexes,MatrixCell>>,MatrixIndexes,MatrixBlock>
+ public static class BinaryCellToBinaryBlockFunction extends CellToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<MatrixIndexes,MatrixCell>>,MatrixIndexes,MatrixBlock>
{
private static final long serialVersionUID = 3928810989462198243L;
@@ -571,83 +591,84 @@ public class RDDConverterUtils {
}
@Override
- public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<MatrixIndexes,MatrixCell>> arg0)
- throws Exception
+ public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<MatrixIndexes,MatrixCell>> arg0)
+ throws Exception
{
ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret = new ArrayList<>();
ReblockBuffer rbuff = new ReblockBuffer(_bufflen, _rlen, _clen, _blen);
-
+
while( arg0.hasNext() )
{
//unpack the binary cell input
Tuple2<MatrixIndexes,MatrixCell> tmp = arg0.next();
-
+
//parse input ijv triple
long row = tmp._1().getRowIndex();
long col = tmp._1().getColumnIndex();
double val = tmp._2().getValue();
-
+
//flush buffer if necessary
if( rbuff.getSize() >= rbuff.getCapacity() )
flushBufferToList(rbuff, ret);
-
+
//add value to reblock buffer
rbuff.appendCell(row, col, val);
}
-
+
//final flush buffer
flushBufferToList(rbuff, ret);
-
+
return ret.iterator();
}
}
-
+
/////////////////////////////////
// CSV-SPECIFIC FUNCTIONS
- private static class CSVAnalysisFunction implements Function<Text,String>
+ private static class CSVAnalysisFunction implements Function<Text,String>
{
private static final long serialVersionUID = 2310303223289674477L;
private final LongAccumulator _aNnz;
private final String _delim;
-
+
public CSVAnalysisFunction( LongAccumulator aNnz, String delim )
{
_aNnz = aNnz;
_delim = delim;
}
-
+
@Override
- public String call(Text v1)
- throws Exception
+ public String call(Text v1)
+ throws Exception
{
//parse input line
String line = v1.toString();
String[] cols = IOUtilFunctions.split(line, _delim);
-
+
//determine number of non-zeros of row (w/o string parsing)
int lnnz = IOUtilFunctions.countNnz(cols);
-
+
//update counters
_aNnz.add( lnnz );
+
return line;
}
-
+
}
/**
* This functions allows to map rdd partitions of csv rows into a set of partial binary blocks.
- *
- * NOTE: For this csv to binary block function, we need to hold all output blocks per partition
- * in-memory. Hence, we keep state of all column blocks and aggregate row segments into these blocks.
+ *
+ * NOTE: For this csv to binary block function, we need to hold all output blocks per partition
+ * in-memory. Hence, we keep state of all column blocks and aggregate row segments into these blocks.
* In terms of memory consumption this is better than creating partial blocks of row segments.
- *
+ *
*/
- private static class CSVToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<Text,Long>>,MatrixIndexes,MatrixBlock>
+ private static class CSVToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<Text,Long>>,MatrixIndexes,MatrixBlock>
{
private static final long serialVersionUID = -4948430402942717043L;
-
+
private long _rlen = -1;
private long _clen = -1;
private int _blen = -1;
@@ -658,7 +679,7 @@ public class RDDConverterUtils {
private boolean _fill = false;
private double _fillValue = 0;
private Set<String> _naStrings;
-
+
public CSVToBinaryBlockFunction(DataCharacteristics mc, boolean sparse, boolean hasHeader, String delim, boolean fill, double fillValue, Set<String> naStrings)
{
_rlen = mc.getRows();
@@ -674,7 +695,7 @@ public class RDDConverterUtils {
}
@Override
- public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<Text,Long>> arg0)
+ public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<Text,Long>> arg0)
throws Exception {
ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret = new ArrayList<>();
@@ -686,14 +707,14 @@ public class RDDConverterUtils {
Tuple2<Text,Long> tmp = arg0.next();
String row = tmp._1().toString();
long rowix = tmp._2() + (_header ? 0 : 1);
-
+
//skip existing header
- if( _header && rowix == 0 )
+ if( _header && rowix == 0 )
continue;
-
+
long rix = UtilFunctions.computeBlockIndex(rowix, _blen);
int pos = UtilFunctions.computeCellInBlock(rowix, _blen);
-
+
//create new blocks for entire row
if( ix[0] == null || ix[0].getRowIndex() != rix ) {
if( ix[0] !=null )
@@ -701,11 +722,11 @@ public class RDDConverterUtils {
long len = UtilFunctions.computeBlockSize(_rlen, rix, _blen);
createBlocks(rowix, (int)len, ix, mb);
}
-
+
//process row data
String[] parts = IOUtilFunctions.split(row, _delim);
boolean emptyFound = false;
- for( int cix=1, pix=0; cix<=ncblks; cix++ )
+ for( int cix=1, pix=0; cix<=ncblks; cix++ )
{
final MatrixBlock mbc = mb[cix-1];
int lclen = UtilFunctions.computeBlockSize(_clen, cix, _blen);
@@ -723,24 +744,24 @@ public class RDDConverterUtils {
mbc.appendValue(pos, j, val);
}
}
-
+
//sanity check empty cells filled w/ values
IOUtilFunctions.checkAndRaiseErrorCSVEmptyField(row, _fill, emptyFound);
}
-
+
//flush last blocks
flushBlocksToList(ix, mb, ret);
-
+
return ret.iterator();
}
-
+
// Creates new state of empty column blocks for current global row index.
private void createBlocks(long rowix, int lrlen, MatrixIndexes[] ix, MatrixBlock[] mb)
{
//compute row block index and number of column blocks
long rix = UtilFunctions.computeBlockIndex(rowix, _blen);
int ncblks = (int)Math.ceil((double)_clen/_blen);
-
+
//create all column blocks (assume dense since csv is dense text format)
for( int cix=1; cix<=ncblks; cix++ ) {
int lclen = UtilFunctions.computeBlockSize(_clen, cix, _blen);
@@ -749,7 +770,7 @@ public class RDDConverterUtils {
mb[cix-1].allocateBlock();
}
}
-
+
// Flushes current state of filled column blocks to output list.
private static void flushBlocksToList( MatrixIndexes[] ix, MatrixBlock[] mb, ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret ) {
int len = ix.length;
@@ -761,17 +782,17 @@ public class RDDConverterUtils {
}
}
- private static class LabeledPointToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<org.apache.spark.mllib.regression.LabeledPoint,Long>>,MatrixIndexes,MatrixBlock>
- {
+ private static class LabeledPointToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<org.apache.spark.mllib.regression.LabeledPoint,Long>>,MatrixIndexes,MatrixBlock>
+ {
private static final long serialVersionUID = 2290124693964816276L;
-
+
private final long _rlen;
private final long _clen;
private final int _blen;
private final boolean _sparseX;
private final boolean _labels;
private final LongAccumulator _aNnz;
-
+
public LabeledPointToBinaryBlockFunction(DataCharacteristics mc, boolean labels, LongAccumulator aNnz) {
_rlen = mc.getRows();
_clen = mc.getCols();
@@ -783,26 +804,26 @@ public class RDDConverterUtils {
}
@Override
- public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<org.apache.spark.mllib.regression.LabeledPoint,Long>> arg0)
- throws Exception
+ public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<org.apache.spark.mllib.regression.LabeledPoint,Long>> arg0)
+ throws Exception
{
ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret = new ArrayList<>();
int ncblks = (int)Math.ceil((double)_clen/_blen);
MatrixIndexes[] ix = new MatrixIndexes[ncblks];
MatrixBlock[] mb = new MatrixBlock[ncblks];
-
+
while( arg0.hasNext() )
{
Tuple2<org.apache.spark.mllib.regression.LabeledPoint,Long> tmp = arg0.next();
org.apache.spark.mllib.regression.LabeledPoint row = tmp._1();
- boolean lsparse = _sparseX || (!_labels &&
+ boolean lsparse = _sparseX || (!_labels &&
row.features() instanceof org.apache.spark.mllib.linalg.SparseVector);
long rowix = tmp._2() + 1;
-
+
long rix = UtilFunctions.computeBlockIndex(rowix, _blen);
int pos = UtilFunctions.computeCellInBlock(rowix, _blen);
-
+
//create new blocks for entire row
if( ix[0] == null || ix[0].getRowIndex() != rix ) {
if( ix[0] !=null )
@@ -810,7 +831,7 @@ public class RDDConverterUtils {
long len = UtilFunctions.computeBlockSize(_rlen, rix, _blen);
createBlocks(rowix, (int)len, ix, mb, lsparse);
}
-
+
//process row data
if( _labels ) {
double val = row.label();
@@ -821,7 +842,7 @@ public class RDDConverterUtils {
int lnnz = row.features().numNonzeros();
if( row.features() instanceof org.apache.spark.mllib.linalg.SparseVector )
{
- org.apache.spark.mllib.linalg.SparseVector srow =
+ org.apache.spark.mllib.linalg.SparseVector srow =
(org.apache.spark.mllib.linalg.SparseVector) row.features();
for( int k=0; k<lnnz; k++ ) {
int gix = srow.indices()[k]+1;
@@ -840,20 +861,20 @@ public class RDDConverterUtils {
_aNnz.add(lnnz);
}
}
-
+
//flush last blocks
flushBlocksToList(ix, mb, ret);
-
+
return ret.iterator();
}
-
+
// Creates new state of empty column blocks for current global row index.
private void createBlocks(long rowix, int lrlen, MatrixIndexes[] ix, MatrixBlock[] mb, boolean lsparse)
{
//compute row block index and number of column blocks
long rix = UtilFunctions.computeBlockIndex(rowix, _blen);
int ncblks = (int)Math.ceil((double)_clen/_blen);
-
+
//create all column blocks (assume dense since csv is dense text format)
for( int cix=1; cix<=ncblks; cix++ ) {
int lclen = UtilFunctions.computeBlockSize(_clen, cix, _blen);
@@ -862,7 +883,7 @@ public class RDDConverterUtils {
mb[cix-1].allocateBlock();
}
}
-
+
// Flushes current state of filled column blocks to output list.
private static void flushBlocksToList( MatrixIndexes[] ix, MatrixBlock[] mb, ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret ) {
int len = ix.length;
@@ -873,68 +894,141 @@ public class RDDConverterUtils {
}
}
}
-
- private static class BinaryBlockToCSVFunction implements FlatMapFunction<Tuple2<MatrixIndexes,MatrixBlock>,String>
+
+ private static class BinaryBlockToCSVFunction implements FlatMapFunction<Tuple2<MatrixIndexes,MatrixBlock>,String>
{
private static final long serialVersionUID = 1891768410987528573L;
private FileFormatPropertiesCSV _props = null;
-
+
public BinaryBlockToCSVFunction(FileFormatPropertiesCSV props) {
_props = props;
}
@Override
public Iterator<String> call(Tuple2<MatrixIndexes, MatrixBlock> arg0)
- throws Exception
+ throws Exception
{
MatrixIndexes ix = arg0._1();
MatrixBlock blk = arg0._2();
-
+
ArrayList<String> ret = new ArrayList<>();
-
+
//handle header information
if(_props.hasHeader() && ix.getRowIndex()==1 ) {
StringBuilder sb = new StringBuilder();
- for(int j = 1; j < blk.getNumColumns(); j++) {
- if(j != 1)
- sb.append(_props.getDelim());
- sb.append("C" + j);
- }
- ret.add(sb.toString());
- }
-
+ for(int j = 1; j < blk.getNumColumns(); j++) {
+ if(j != 1)
+ sb.append(_props.getDelim());
+ sb.append("C" + j);
+ }
+ ret.add(sb.toString());
+ }
+
//handle matrix block data
StringBuilder sb = new StringBuilder();
- for(int i=0; i<blk.getNumRows(); i++) {
- for(int j=0; j<blk.getNumColumns(); j++) {
- if(j != 0)
- sb.append(_props.getDelim());
- double val = blk.quickGetValue(i, j);
- if(!(_props.isSparse() && val == 0))
- sb.append(val);
+ for(int i=0; i<blk.getNumRows(); i++) {
+ for(int j=0; j<blk.getNumColumns(); j++) {
+ if(j != 0)
+ sb.append(_props.getDelim());
+ double val = blk.quickGetValue(i, j);
+ if(!(_props.isSparse() && val == 0))
+ sb.append(val);
}
- ret.add(sb.toString());
- sb.setLength(0); //reset
- }
-
+ ret.add(sb.toString());
+ sb.setLength(0); //reset
+ }
+
return ret.iterator();
}
}
- private static class SliceBinaryBlockToRowsFunction implements PairFlatMapFunction<Tuple2<MatrixIndexes,MatrixBlock>,Long,Tuple2<Long,MatrixBlock>>
+ private static class BinaryBlockToLIBSVMFunction
+ implements FlatMapFunction<Tuple2<MatrixIndexes, MatrixBlock>, String> {
+ private static final long serialVersionUID = 1891768410987528573L;
+ private FileFormatPropertiesLIBSVM _props = null;
+
+ public BinaryBlockToLIBSVMFunction(FileFormatPropertiesLIBSVM props) {
+ _props = props;
+ }
+
+ // Return string in libsvm format (<index#>:<value#>)
+ protected void appendIndexValLibsvm(StringBuilder sb, int index, double value) {
+ sb.append(index + 1); // convert 0 based matrix index to 1 base libsvm index
+ sb.append(_props.getIndexDelim());
+ sb.append(value);
+ }
+
+ @Override
+ public Iterator<String> call(Tuple2<MatrixIndexes, MatrixBlock> arg0) throws Exception {
+ MatrixBlock blk = arg0._2();
+ ArrayList<String> ret = new ArrayList<>();
+ StringBuilder sb = new StringBuilder();
+ boolean sparse = blk.isInSparseFormat();
+
+ // Write data lines
+ if(sparse) //SPARSE
+ {
+ SparseBlock sblock = blk.getSparseBlock();
+ for(int i = 0; i < blk.getNumRows(); i++) {
+ // append the class label as the 1st column
+ double label = (sblock != null) ? sblock.get(i, blk.getNumColumns() - 1) : 0;
+ sb.append(label);
+
+ if(sblock != null && i < sblock.numRows() && !sblock.isEmpty(i)) {
+ int pos = sblock.pos(i);
+ int alen = sblock.size(i);
+ int[] aix = sblock.indexes(i);
+ double[] avals = sblock.values(i);
+ // append sparse row
+ for(int k = pos; k < pos + alen; k++) {
+ if(aix[k] != blk.getNumColumns() - 1) {
+ sb.append(_props.getDelim());
+ appendIndexValLibsvm(sb, aix[k], avals[k]);
+ }
+ }
+ }
+
+ ret.add(sb.toString());
+ sb.setLength(0);
+ }
+ }
+ else //DENSE
+ {
+ for(int i = 0; i < blk.getNumRows(); i++) {
+ // append the class label as the 1st column
+ double label = blk.quickGetValue(i, blk.getNumColumns() - 1);
+ sb.append(label);
+
+ // append dense row
+ for(int j = 0; j < blk.getNumColumns() - 1; j++) {
+ double val = blk.quickGetValue(i, j);
+ if(val != 0) {
+ sb.append(_props.getDelim());
+ appendIndexValLibsvm(sb, j, val);
+ }
+ }
+ ret.add(sb.toString());
+ sb.setLength(0);
+ }
+ }
+ return ret.iterator();
+ }
+ }
+
+ private static class SliceBinaryBlockToRowsFunction implements PairFlatMapFunction<Tuple2<MatrixIndexes,MatrixBlock>,Long,Tuple2<Long,MatrixBlock>>
{
private static final long serialVersionUID = 7192024840710093114L;
-
+
private int _blen = -1;
-
+
public SliceBinaryBlockToRowsFunction(int blen) {
_blen = blen;
}
-
+
@Override
- public Iterator<Tuple2<Long,Tuple2<Long,MatrixBlock>>> call(Tuple2<MatrixIndexes, MatrixBlock> arg0)
- throws Exception
+ public Iterator<Tuple2<Long,Tuple2<Long,MatrixBlock>>> call(Tuple2<MatrixIndexes, MatrixBlock> arg0)
+ throws Exception
{
ArrayList<Tuple2<Long,Tuple2<Long,MatrixBlock>>> ret = new ArrayList<>();
MatrixIndexes ix = arg0._1();
@@ -946,26 +1040,26 @@ public class RDDConverterUtils {
}
return ret.iterator();
}
-
+
}
private static class ConcatenateBlocksFunction implements PairFunction<Tuple2<Long, Iterable<Tuple2<Long,MatrixBlock>>>,MatrixIndexes,MatrixBlock>
{
private static final long serialVersionUID = -7879603125149650097L;
-
+
private long _clen = -1;
private int _blen = -1;
private int _ncblks = -1;
-
+
public ConcatenateBlocksFunction(long clen, int blen) {
_clen = clen;
_blen = blen;
_ncblks = (int)Math.ceil((double)clen/blen);
}
-
+
@Override
public Tuple2<MatrixIndexes, MatrixBlock> call(Tuple2<Long,Iterable<Tuple2<Long, MatrixBlock>>> arg0)
- throws Exception
+ throws Exception
{
long rowIndex = arg0._1();
MatrixBlock[] tmpBlks = new MatrixBlock[_ncblks];
@@ -988,10 +1082,10 @@ public class RDDConverterUtils {
/////////////////////////////////
// DATAFRAME-SPECIFIC FUNCTIONS
- private static class DataFrameToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<Row,Long>>,MatrixIndexes,MatrixBlock>
+ private static class DataFrameToBinaryBlockFunction implements PairFlatMapFunction<Iterator<Tuple2<Row,Long>>,MatrixIndexes,MatrixBlock>
{
private static final long serialVersionUID = 653447740362447236L;
-
+
private long _rlen = -1;
private long _clen = -1;
private int _blen = -1;
@@ -999,7 +1093,7 @@ public class RDDConverterUtils {
private boolean _sparse = false;
private boolean _containsID;
private boolean _isVector;
-
+
public DataFrameToBinaryBlockFunction(DataCharacteristics mc, boolean sparse, boolean containsID, boolean isVector) {
_rlen = mc.getRows();
_clen = mc.getCols();
@@ -1009,25 +1103,25 @@ public class RDDConverterUtils {
_containsID = containsID;
_isVector = isVector;
}
-
+
@Override
- public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<Row, Long>> arg0)
- throws Exception
+ public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<Row, Long>> arg0)
+ throws Exception
{
ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret = new ArrayList<>();
-
+
int ncblks = (int)Math.ceil((double)_clen/_blen);
MatrixIndexes[] ix = new MatrixIndexes[ncblks];
MatrixBlock[] mb = new MatrixBlock[ncblks];
-
+
while( arg0.hasNext() )
{
Tuple2<Row,Long> tmp = arg0.next();
long rowix = tmp._2() + 1;
-
+
long rix = UtilFunctions.computeBlockIndex(rowix, _blen);
int pos = UtilFunctions.computeCellInBlock(rowix, _blen);
-
+
//create new blocks for entire row
if( ix[0] == null || ix[0].getRowIndex() != rix ) {
if( ix[0] !=null )
@@ -1035,7 +1129,7 @@ public class RDDConverterUtils {
long len = UtilFunctions.computeBlockSize(_rlen, rix, _blen);
createBlocks(rowix, (int)len, ix, mb);
}
-
+
//process row data
int off = _containsID ? 1 : 0;
Object obj = _isVector ? tmp._1().get(off) : tmp._1();
@@ -1070,20 +1164,20 @@ public class RDDConverterUtils {
}
}
}
-
+
//flush last blocks
flushBlocksToList(ix, mb, ret);
-
+
return ret.iterator();
}
-
+
// Creates new state of empty column blocks for current global row index.
private void createBlocks(long rowix, int lrlen, MatrixIndexes[] ix, MatrixBlock[] mb)
{
//compute row block index and number of column blocks
long rix = UtilFunctions.computeBlockIndex(rowix, _blen);
int ncblks = (int)Math.ceil((double)_clen/_blen);
-
+
//create all column blocks (assume dense since csv is dense text format)
for( int cix=1; cix<=ncblks; cix++ ) {
int lclen = UtilFunctions.computeBlockSize(_clen, cix, _blen);
@@ -1092,7 +1186,7 @@ public class RDDConverterUtils {
mb[cix-1].allocateBlock();
}
}
-
+
// Flushes current state of filled column blocks to output list.
private static void flushBlocksToList( MatrixIndexes[] ix, MatrixBlock[] mb, ArrayList<Tuple2<MatrixIndexes,MatrixBlock>> ret ) {
int len = ix.length;
@@ -1104,14 +1198,14 @@ public class RDDConverterUtils {
}
}
- private static class DataFrameAnalysisFunction implements Function<Row,Row>
- {
+ private static class DataFrameAnalysisFunction implements Function<Row,Row>
+ {
private static final long serialVersionUID = 5705371332119770215L;
-
+
private LongAccumulator _aNnz = null;
private boolean _containsID;
private boolean _isVector;
-
+
public DataFrameAnalysisFunction( LongAccumulator aNnz, boolean containsID, boolean isVector) {
_aNnz = aNnz;
_containsID = containsID;
@@ -1124,19 +1218,19 @@ public class RDDConverterUtils {
int off = _containsID ? 1 : 0;
Object vect = _isVector ? arg0.get(off) : arg0;
int lnnz = countNnz(vect, _isVector, off);
-
+
//update counters
_aNnz.add( lnnz );
return arg0;
}
}
- public static class DataFrameExtractIDFunction implements PairFunction<Row, Row,Long>
+ public static class DataFrameExtractIDFunction implements PairFunction<Row, Row,Long>
{
private static final long serialVersionUID = 7438855241666363433L;
private int _index = -1;
-
+
public DataFrameExtractIDFunction(int index) {
_index = index;
}
@@ -1146,7 +1240,7 @@ public class RDDConverterUtils {
//extract 1-based IDs and convert to 0-based positions
long id = UtilFunctions.toLong(UtilFunctions.getDouble(arg0.get(_index)));
if( id <= 0 ) {
- throw new DMLRuntimeException("ID Column '" + DF_ID_COLUMN
+ throw new DMLRuntimeException("ID Column '" + DF_ID_COLUMN
+ "' expected to be 1-based, but found value: "+id);
}
return new Tuple2<>(arg0, id-1);
@@ -1154,13 +1248,13 @@ public class RDDConverterUtils {
}
private static class ConvertRowBlocksToRows implements Function<Tuple2<Long, Iterable<Tuple2<Long, MatrixBlock>>>, Row> {
-
+
private static final long serialVersionUID = 4441184411670316972L;
-
+
private int _clen;
private int _blen;
private boolean _toVector;
-
+
public ConvertRowBlocksToRows(int clen, int blen, boolean toVector) {
_clen = clen;
_blen = blen;
@@ -1169,11 +1263,11 @@ public class RDDConverterUtils {
@Override
public Row call(Tuple2<Long, Iterable<Tuple2<Long, MatrixBlock>>> arg0)
- throws Exception
+ throws Exception
{
Object[] row = new Object[_toVector ? 2 : _clen+1];
row[0] = (double) arg0._1(); //row index
-
+
//copy block data into target row
if( _toVector ) {
if( _clen <= _blen ) { //single block
@@ -1197,8 +1291,132 @@ public class RDDConverterUtils {
row[cl+j+1] = mb.quickGetValue(0, j);
}
}
-
+
return RowFactory.create(row);
}
}
+
+ ////////////////////////////////
+ // LIBSVM FUNCTIONS
+
+ // LIBSVM-SPECIFIC FUNCTIONS
+ private static class LIBSVMAnalysisFunction implements Function<Text, String> {
+ private static final long serialVersionUID = 2310303223289674477L;
+
+ private LongAccumulator _aNnz = null;
+ private String _delim = null;
+ private String _indexDelim = null;
+
+ public LIBSVMAnalysisFunction(LongAccumulator aNnz, String delim, String indexDelim) {
+ _delim = delim;
+ _indexDelim = indexDelim;
+ _aNnz = aNnz;
+ }
+
+ @Override public String call(Text v1) throws Exception {
+ //parse row w/ first entry being the label
+ String line = v1.toString();
+ String[] parts = line.split(_delim);
+ String[] cols = new String[parts.length];
+ cols[0] = parts[0];
+
+ //parse entire row
+ for(int i = 1; i < parts.length; i++) {
+ String[] pair = parts[i].split(_indexDelim);
+ cols[i] = pair[1];
+ }
+ //determine number of non-zeros of row (w/o string parsing)
+ int lnnz = IOUtilFunctions.countNnz(cols);
+
+ _aNnz.add(lnnz);
+ return line;
+ }
+ }
+
+ /**
+ * This functions allows to map rdd partitions of libsvm rows into a set of partial binary blocks.
+ */
+ private static class LIBSVMToBinaryBlockFunction extends CellToBinaryBlockFunction
+ implements PairFlatMapFunction<Iterator<Tuple2<Text, Long>>, MatrixIndexes, MatrixBlock>
+ {
+ private static final long serialVersionUID = -4948430402942717043L;
+ private String _delim;
+ private String _indexDelim;
+
+ protected LIBSVMToBinaryBlockFunction(DataCharacteristics mc, String delim, String indexDelim) {
+ super(mc);
+ _delim = delim;
+ _indexDelim = indexDelim;
+}
+
+ @Override
+ public Iterator<Tuple2<MatrixIndexes, MatrixBlock>> call(Iterator<Tuple2<Text, Long>> arg0) throws Exception {
+ ArrayList<Tuple2<MatrixIndexes, MatrixBlock>> ret = new ArrayList<>();
+ ReblockBuffer rbuff = new ReblockBuffer(_bufflen, _rlen, _clen, _blen);
+
+ while(arg0.hasNext()) {
+ Tuple2<Text, Long> tmp = arg0.next();
+ String rowStr = tmp._1().toString();
+ long rowix = tmp._2() + 1;
+
+ //parse input libsvm row
+ String[] parts = rowStr.split(_delim);
+ double label = Double.parseDouble(parts[0]);
+
+ //parse entire row
+ for(int i = 1; i < parts.length; i++) {
+ String[] pair = parts[i].split(_indexDelim);
+ long col = Integer.parseInt(pair[0]);
+ double val = Double.parseDouble(pair[1]);
+
+ //flush buffer if necessary
+ if(rbuff.getSize() >= rbuff.getCapacity())
+ flushBufferToList(rbuff, ret);
+
+ rbuff.appendCell(rowix, col, val);
+ }
+
+ // flush buffer if necessary
+ if(rbuff.getSize() >= rbuff.getCapacity())
+ flushBufferToList(rbuff, ret);
+ // write <row>:<clen>:<label>
+ rbuff.appendCell(rowix, _clen, label);
+ }
+
+ //final flush buffer
+ flushBufferToList(rbuff, ret);
+
+ return ret.iterator();
+ }
+ }
+
+ public static JavaPairRDD<MatrixIndexes, MatrixBlock> libsvmToBinaryBlock(JavaSparkContext sc,
+ JavaPairRDD<LongWritable, Text> input, DataCharacteristics mc, String delim, String indexDelim) {
+
+ //determine unknown dimensions and sparsity if required
+ //(w/ robustness for mistakenly counted header in nnz)
+ if(!mc.dimsKnown(true)) {
+ LongAccumulator aNnz = sc.sc().longAccumulator("nnz");
+ JavaRDD<String> tmp = input.values().map(new LIBSVMAnalysisFunction(aNnz, delim, indexDelim));
+ long rlen = tmp.count();
+ long nnz = UtilFunctions.toLong(aNnz.value()); //Math.min(rlen * clen, UtilFunctions.toLong(aNnz.value()));
+ mc.set(rlen, mc.getCols(), mc.getBlocksize(), nnz);
+ }
+
+ //prepare libsvm w/ row indexes (sorted by filenames)
+ JavaPairRDD<Text, Long> prepinput = input.values().zipWithIndex(); //zip row index
+
+ //convert libsvm rdd to binary block rdd (w/ partial blocks)
+ JavaPairRDD<MatrixIndexes, MatrixBlock> out = prepinput
+ .mapPartitionsToPair(new LIBSVMToBinaryBlockFunction(mc, delim, indexDelim));
+
+ //aggregate partial matrix blocks (w/ preferred number of output
+ //partitions as the data is likely smaller in binary block format,
+ //but also to bound the size of partitions for compressed inputs)
+ int parts = SparkUtils.getNumPreferredPartitions(mc, out);
+ return RDDAggregateUtils.mergeByKey(out, parts, false);
+ }
+ ///////////////////////////////
+ // END LIBSVM FUNCTIONS
}
+
diff --git a/src/main/java/org/apache/sysds/runtime/io/FileFormatPropertiesLIBSVM.java b/src/main/java/org/apache/sysds/runtime/io/FileFormatPropertiesLIBSVM.java
new file mode 100644
index 0000000..7ebb455
--- /dev/null
+++ b/src/main/java/org/apache/sysds/runtime/io/FileFormatPropertiesLIBSVM.java
@@ -0,0 +1,89 @@
+/*
+ * 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.io;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.sysds.parser.DataExpression;
+
+import java.io.Serializable;
+
+public class FileFormatPropertiesLIBSVM extends FileFormatProperties implements Serializable {
+ protected static final Log LOG = LogFactory.getLog(FileFormatPropertiesLIBSVM.class.getName());
+ private static final long serialVersionUID = -2870393360885401604L;
+
+ private String delim;
+ private String indexDelim;
+ private boolean sparse;
+
+ public FileFormatPropertiesLIBSVM() {
+ // get the default values for LIBSVM properties from the language layer
+ this.delim = DataExpression.DEFAULT_DELIM_DELIMITER;
+ this.indexDelim = DataExpression.DEFAULT_LIBSVM_INDEX_DELIM;
+
+ if(LOG.isDebugEnabled())
+ LOG.debug("FileFormatPropertiesLIBSVM: " + this.toString());
+ }
+
+ public FileFormatPropertiesLIBSVM(String delim, String indexDelim) {
+ this();
+ this.delim = delim;
+ this.indexDelim = indexDelim;
+ if(LOG.isDebugEnabled())
+ LOG.debug("FileFormatPropertiesLIBSVM full settings: " + this.toString());
+ }
+
+ public FileFormatPropertiesLIBSVM(String delim, String indexDelim, boolean sparse) {
+ this();
+ this.delim = delim;
+ this.indexDelim = indexDelim;
+ this.sparse = sparse;
+ if(LOG.isDebugEnabled())
+ LOG.debug("FileFormatPropertiesLIBSVM full settings: " + this.toString());
+ }
+
+ public String getDelim() {
+ return delim;
+ }
+
+ public void setDelim(String delim) {
+ this.delim = delim;
+ }
+
+ public String getIndexDelim() {
+ return indexDelim;
+ }
+
+ public void setIndexDelim(String indexDelim) {
+ this.indexDelim = indexDelim;
+ }
+
+ public boolean isSparse() {
+ return sparse;
+ }
+
+ @Override public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append(" delim " + delim);
+ sb.append(" indexDelim " + indexDelim);
+ sb.append(" sparse " + sparse);
+ return sb.toString();
+ }
+}
diff --git a/src/main/java/org/apache/sysds/runtime/io/MatrixReaderFactory.java b/src/main/java/org/apache/sysds/runtime/io/MatrixReaderFactory.java
index 65df7d8..48573f8 100644
--- a/src/main/java/org/apache/sysds/runtime/io/MatrixReaderFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/io/MatrixReaderFactory.java
@@ -38,30 +38,30 @@ public class MatrixReaderFactory {
if (LOG.isDebugEnabled()){
LOG.debug("reading parallel: " + par + " mcsr: " + mcsr);
}
-
+
switch(fmt) {
case TEXT:
case MM:
reader = (par & mcsr) ?
new ReaderTextCellParallel(fmt) : new ReaderTextCell(fmt);
break;
-
+
case CSV:
- reader = (par & mcsr) ?
+ reader = (par & mcsr) ?
new ReaderTextCSVParallel(new FileFormatPropertiesCSV()) :
new ReaderTextCSV(new FileFormatPropertiesCSV());
break;
-
+
case LIBSVM:
- reader = (par & mcsr) ?
- new ReaderTextLIBSVMParallel() : new ReaderTextLIBSVM();
+ reader = (par & mcsr) ? new ReaderTextLIBSVMParallel(
+ new FileFormatPropertiesLIBSVM()) : new ReaderTextLIBSVM(new FileFormatPropertiesLIBSVM());
break;
-
+
case BINARY:
- reader = (par & mcsr) ?
+ reader = (par & mcsr) ?
new ReaderBinaryBlockParallel(false) : new ReaderBinaryBlock(false);
break;
-
+
default:
throw new DMLRuntimeException("Failed to create matrix reader for unknown format: " + fmt.toString());
}
@@ -72,12 +72,12 @@ public class MatrixReaderFactory {
//check valid read properties
if( props == null )
throw new DMLRuntimeException("Failed to create matrix reader with empty properties.");
-
+
MatrixReader reader = null;
FileFormat fmt = props.fmt;
boolean par = ConfigurationManager.getCompilerConfigFlag(ConfigType.PARALLEL_CP_READ_TEXTFORMATS);
boolean mcsr = MatrixBlock.DEFAULT_SPARSEBLOCK == SparseBlock.Type.MCSR;
-
+
if (LOG.isDebugEnabled()){
LOG.debug("reading parallel: " + par + " mcsr: " + mcsr);
}
@@ -88,25 +88,26 @@ public class MatrixReaderFactory {
reader = (par & mcsr) ?
new ReaderTextCellParallel(fmt) : new ReaderTextCell(fmt);
break;
-
+
case CSV:
reader = (par & mcsr) ?
new ReaderTextCSVParallel( props.formatProperties!=null ?
(FileFormatPropertiesCSV)props.formatProperties : new FileFormatPropertiesCSV()) :
- new ReaderTextCSV( props.formatProperties!=null ?
+ new ReaderTextCSV( props.formatProperties!=null ?
(FileFormatPropertiesCSV)props.formatProperties : new FileFormatPropertiesCSV());
break;
-
+
case LIBSVM:
- reader = (par & mcsr) ?
- new ReaderTextLIBSVMParallel() : new ReaderTextLIBSVM();
+ FileFormatPropertiesLIBSVM fileFormatPropertiesLIBSVM = props.formatProperties != null ? (FileFormatPropertiesLIBSVM) props.formatProperties : new FileFormatPropertiesLIBSVM();
+ reader = (par & mcsr) ? new ReaderTextLIBSVMParallel(fileFormatPropertiesLIBSVM) : new ReaderTextLIBSVM(
+ fileFormatPropertiesLIBSVM);
break;
-
+
case BINARY:
reader = (par & mcsr) ?
new ReaderBinaryBlockParallel(props.localFS) : new ReaderBinaryBlock(props.localFS);
break;
-
+
default:
throw new DMLRuntimeException("Failed to create matrix reader for unknown format: " + fmt.toString());
}
diff --git a/src/main/java/org/apache/sysds/runtime/io/MatrixWriterFactory.java b/src/main/java/org/apache/sysds/runtime/io/MatrixWriterFactory.java
index 5fd237b..2dfd588 100644
--- a/src/main/java/org/apache/sysds/runtime/io/MatrixWriterFactory.java
+++ b/src/main/java/org/apache/sysds/runtime/io/MatrixWriterFactory.java
@@ -24,17 +24,17 @@ import org.apache.sysds.common.Types.FileFormat;
import org.apache.sysds.conf.CompilerConfig.ConfigType;
import org.apache.sysds.runtime.DMLRuntimeException;
-public class MatrixWriterFactory
+public class MatrixWriterFactory
{
public static MatrixWriter createMatrixWriter(FileFormat fmt) {
return createMatrixWriter(fmt, -1, null);
}
- public static MatrixWriter createMatrixWriter(FileFormat fmt, int replication, FileFormatProperties props)
+ public static MatrixWriter createMatrixWriter(FileFormat fmt, int replication, FileFormatProperties props)
{
MatrixWriter writer = null;
-
+
switch(fmt) {
case TEXT:
if( ConfigurationManager.getCompilerConfigFlag(ConfigType.PARALLEL_CP_WRITE_TEXTFORMATS) )
@@ -42,16 +42,16 @@ public class MatrixWriterFactory
else
writer = new WriterTextCell();
break;
-
+
case MM:
//note: disabled parallel cp write of matrix market in order to ensure the
//requirement of writing out a single file
-
+
//if( OptimizerUtils.PARALLEL_CP_WRITE_TEXTFORMATS )
// writer = new WriterMatrixMarketParallel();
writer = new WriterMatrixMarket();
break;
-
+
case CSV:
if( props!=null && !(props instanceof FileFormatPropertiesCSV) )
throw new DMLRuntimeException("Wrong type of file format properties for CSV writer.");
@@ -60,25 +60,27 @@ public class MatrixWriterFactory
else
writer = new WriterTextCSV((FileFormatPropertiesCSV)props);
break;
-
+
case LIBSVM:
+ if(props != null && !(props instanceof FileFormatPropertiesLIBSVM))
+ throw new DMLRuntimeException("Wrong type of file format properties for LIBSVM writer.");
if( ConfigurationManager.getCompilerConfigFlag(ConfigType.PARALLEL_CP_WRITE_TEXTFORMATS) )
- writer = new WriterTextLIBSVMParallel();
+ writer = new WriterTextLIBSVMParallel((FileFormatPropertiesLIBSVM) props);
else
- writer = new WriterTextLIBSVM();
+ writer = new WriterTextLIBSVM((FileFormatPropertiesLIBSVM) props);
break;
-
+
case BINARY:
if( ConfigurationManager.getCompilerConfigFlag(ConfigType.PARALLEL_CP_WRITE_BINARYFORMATS) )
writer = new WriterBinaryBlockParallel(replication);
else
writer = new WriterBinaryBlock(replication);
break;
-
+
default:
throw new DMLRuntimeException("Failed to create matrix writer for unknown format: " + fmt.toString());
}
-
+
return writer;
}
}
diff --git a/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVM.java b/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVM.java
index 15eef81..7c83fe4 100644
--- a/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVM.java
+++ b/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVM.java
@@ -37,61 +37,61 @@ import org.apache.sysds.runtime.DMLRuntimeException;
import org.apache.sysds.runtime.data.SparseRowVector;
import org.apache.sysds.runtime.matrix.data.MatrixBlock;
-public class ReaderTextLIBSVM extends MatrixReader
-{
- public ReaderTextLIBSVM() {
+public class ReaderTextLIBSVM extends MatrixReader {
+ private static FileFormatPropertiesLIBSVM _props;
+ public ReaderTextLIBSVM(FileFormatPropertiesLIBSVM props) {
+ _props = props;
}
@Override
- public MatrixBlock readMatrixFromHDFS(String fname, long rlen, long clen, int blen, long estnnz)
- throws IOException, DMLRuntimeException
+ public MatrixBlock readMatrixFromHDFS(String fname, long rlen, long clen, int blen, long estnnz)
+ throws IOException, DMLRuntimeException
{
//allocate output matrix block
MatrixBlock ret = null;
if( rlen>=0 && clen>=0 ) //otherwise allocated on read
ret = createOutputMatrixBlock(rlen, clen, (int)rlen, estnnz, true, false);
-
+
//prepare file access
JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
Path path = new Path( fname );
FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-
+
//check existence and non-empty file
- checkValidInputFile(fs, path);
-
- //core read
+ checkValidInputFile(fs, path);
+
+ //core read
ret = readLIBSVMMatrixFromHDFS(path, job, fs, ret, rlen, clen, blen);
-
+
//finally check if change of sparse/dense block representation required
//(nnz explicitly maintained during read)
ret.examSparsity();
-
+
return ret;
}
-
+
@Override
- public MatrixBlock readMatrixFromInputStream(InputStream is, long rlen, long clen, int blen, long estnnz)
- throws IOException, DMLRuntimeException
+ public MatrixBlock readMatrixFromInputStream(InputStream is, long rlen, long clen, int blen, long estnnz)
+ throws IOException, DMLRuntimeException
{
//allocate output matrix block
MatrixBlock ret = createOutputMatrixBlock(rlen, clen, (int)rlen, estnnz, true, false);
-
- //core read
+
+ //core read
long lnnz = readLIBSVMMatrixFromInputStream(is, "external inputstream", ret,
new MutableInt(0), rlen, clen, blen);
-
+
//finally check if change of sparse/dense block representation required
ret.setNonZeros( lnnz );
ret.examSparsity();
-
+
return ret;
}
-
+
@SuppressWarnings("unchecked")
- private static MatrixBlock readLIBSVMMatrixFromHDFS( Path path, JobConf job, FileSystem fs, MatrixBlock dest,
- long rlen, long clen, int blen)
- throws IOException, DMLRuntimeException
+ private static MatrixBlock readLIBSVMMatrixFromHDFS( Path path, JobConf job, FileSystem fs, MatrixBlock dest,
+ long rlen, long clen, int blen) throws IOException, DMLRuntimeException
{
//prepare file paths in alphanumeric order
ArrayList<Path> files=new ArrayList<>();
@@ -102,13 +102,13 @@ public class ReaderTextLIBSVM extends MatrixReader
}
else
files.add(path);
-
+
//determine matrix size via additional pass if required
if ( dest == null ) {
dest = computeLIBSVMSize(files, clen, job, fs);
clen = dest.getNumColumns();
}
-
+
//actual read of individual files
long lnnz = 0;
MutableInt row = new MutableInt(0);
@@ -116,14 +116,14 @@ public class ReaderTextLIBSVM extends MatrixReader
lnnz += readLIBSVMMatrixFromInputStream(fs.open(files.get(fileNo)),
path.toString(), dest, row, rlen, clen, blen);
}
-
+
//post processing
dest.setNonZeros( lnnz );
-
+
return dest;
}
-
- private static long readLIBSVMMatrixFromInputStream( InputStream is, String srcInfo, MatrixBlock dest, MutableInt rowPos,
+
+ private static long readLIBSVMMatrixFromInputStream( InputStream is, String srcInfo, MatrixBlock dest, MutableInt rowPos,
long rlen, long clen, int blen )
throws IOException
{
@@ -131,48 +131,50 @@ public class ReaderTextLIBSVM extends MatrixReader
String value = null;
int row = rowPos.intValue();
long lnnz = 0;
-
+
// Read the data
try( BufferedReader br = new BufferedReader(new InputStreamReader(is)) ) {
while( (value=br.readLine())!=null ) { //for each line
String rowStr = value.toString().trim();
- lnnz += ReaderTextLIBSVM.parseLibsvmRow(rowStr, vect, (int)clen);
+ lnnz += ReaderTextLIBSVM.parseLibsvmRow(rowStr, vect,
+ (int) clen, _props.getDelim(), _props.getIndexDelim());
dest.appendRow(row, vect);
row++;
}
}
-
+
rowPos.setValue(row);
return lnnz;
}
- private static MatrixBlock computeLIBSVMSize(List<Path> files, long ncol, JobConf job, FileSystem fs)
- throws IOException, DMLRuntimeException
+ private static MatrixBlock computeLIBSVMSize(List<Path> files, long ncol, JobConf job, FileSystem fs)
+ throws IOException, DMLRuntimeException
{
- int nrow = -1;
+ int nrow = 0;
for(int fileNo=0; fileNo<files.size(); fileNo++) {
try( BufferedReader br = new BufferedReader(new InputStreamReader(fs.open(files.get(fileNo)))) ) {
while( br.readLine() != null )
nrow++;
}
}
-
- // allocate target matrix block based on given size;
+
+ // allocate target matrix block based on given size;
return createOutputMatrixBlock(nrow, ncol, nrow, nrow*ncol, true, false);
}
-
- protected static int parseLibsvmRow(String rowStr, SparseRowVector vect, int clen) {
+
+ protected static int parseLibsvmRow(String rowStr, SparseRowVector vect, int clen, String delim,
+ String indexDelim) {
// reset row buffer (but keep allocated arrays)
vect.setSize(0);
-
+
//parse row w/ first entry being the label
- String[] parts = rowStr.split(IOUtilFunctions.LIBSVM_DELIM);
+ String[] parts = rowStr.split(delim);
double label = Double.parseDouble(parts[0]);
-
+
//parse entire row
for( int i=1; i<parts.length; i++ ) {
//parse non-zero: <index#>:<value#>
- String[] pair = parts[i].split(IOUtilFunctions.LIBSVM_INDEX_DELIM);
+ String[] pair = parts[i].split(indexDelim);
vect.append(Integer.parseInt(pair[0])-1, Double.parseDouble(pair[1]));
}
vect.append(clen-1, label);
diff --git a/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVMParallel.java b/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVMParallel.java
index 76fff10..0567ea4 100644
--- a/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVMParallel.java
+++ b/src/main/java/org/apache/sysds/runtime/io/ReaderTextLIBSVMParallel.java
@@ -42,25 +42,26 @@ import org.apache.sysds.runtime.data.SparseRowVector;
import org.apache.sysds.runtime.matrix.data.MatrixBlock;
import org.apache.sysds.runtime.util.CommonThreadPool;
-public class ReaderTextLIBSVMParallel extends MatrixReader
-{
+public class ReaderTextLIBSVMParallel extends MatrixReader {
+ private static FileFormatPropertiesLIBSVM _props;
private int _numThreads = 1;
private SplitOffsetInfos _offsets = null;
- public ReaderTextLIBSVMParallel() {
+ public ReaderTextLIBSVMParallel(FileFormatPropertiesLIBSVM props) {
_numThreads = OptimizerUtils.getParallelTextReadParallelism();
+ _props = props;
}
-
+
@Override
public MatrixBlock readMatrixFromHDFS(String fname, long rlen, long clen,
- int blen, long estnnz)
- throws IOException, DMLRuntimeException
+ int blen, long estnnz)
+ throws IOException, DMLRuntimeException
{
// prepare file access
JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
Path path = new Path(fname);
FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-
+
FileInputFormat.addInputPath(job, path);
TextInputFormat informat = new TextInputFormat();
informat.configure(job);
@@ -79,7 +80,7 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
// Second Read Pass (read, parse strings, append to matrix block)
readLIBSVMMatrixFromHDFS(splits, path, job, ret, rlen, clen, blen);
-
+
//post-processing (representation-specific, change of sparse/dense block representation)
// - nnz explicitly maintained in parallel for the individual splits
ret.examSparsity();
@@ -91,19 +92,18 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
return ret;
}
-
+
@Override
- public MatrixBlock readMatrixFromInputStream(InputStream is, long rlen, long clen, int blen, long estnnz)
- throws IOException, DMLRuntimeException
+ public MatrixBlock readMatrixFromInputStream(InputStream is, long rlen, long clen, int blen, long estnnz)
+ throws IOException, DMLRuntimeException
{
//not implemented yet, fallback to sequential reader
- return new ReaderTextLIBSVM()
- .readMatrixFromInputStream(is, rlen, clen, blen, estnnz);
+ return new ReaderTextLIBSVM(_props).readMatrixFromInputStream(is, rlen, clen, blen, estnnz);
}
-
- private void readLIBSVMMatrixFromHDFS(InputSplit[] splits, Path path, JobConf job,
- MatrixBlock dest, long rlen, long clen, int blen)
- throws IOException
+
+ private void readLIBSVMMatrixFromHDFS(InputSplit[] splits, Path path, JobConf job,
+ MatrixBlock dest, long rlen, long clen, int blen)
+ throws IOException
{
FileInputFormat.addInputPath(job, path);
TextInputFormat informat = new TextInputFormat();
@@ -111,7 +111,7 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
ExecutorService pool = CommonThreadPool.get(_numThreads);
- try
+ try
{
// create read tasks for all splits
ArrayList<LIBSVMReadTask> tasks = new ArrayList<>();
@@ -132,25 +132,25 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
}
}
dest.setNonZeros(lnnz);
- }
+ }
catch (Exception e) {
throw new IOException("Threadpool issue, while parallel read.", e);
}
}
-
+
private MatrixBlock computeLIBSVMSizeAndCreateOutputMatrixBlock(InputSplit[] splits, Path path,
JobConf job, long rlen, long clen, long estnnz)
- throws IOException, DMLRuntimeException
+ throws IOException, DMLRuntimeException
{
int nrow = 0;
int ncol = (int) clen;
-
+
FileInputFormat.addInputPath(job, path);
TextInputFormat informat = new TextInputFormat();
informat.configure(job);
// count rows in parallel per split
- try
+ try
{
ExecutorService pool = CommonThreadPool.get(_numThreads);
ArrayList<CountRowsTask> tasks = new ArrayList<>();
@@ -170,11 +170,11 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
_offsets.setLenghtPerSplit(tasks.indexOf(rt), rt.getRowCount());
nrow = nrow + rt.getRowCount();
}
- }
+ }
catch (Exception e) {
throw new IOException("Threadpool Error " + e.getMessage(), e);
}
-
+
//robustness for wrong dimensions which are already compiled into the plan
if( (rlen != -1 && nrow != rlen) || (clen != -1 && ncol != clen) ) {
String msg = "Read matrix dimensions differ from meta data: ["+nrow+"x"+ncol+"] vs. ["+rlen+"x"+clen+"].";
@@ -189,13 +189,13 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
ncol = (int) clen;
}
}
-
- // allocate target matrix block based on given size;
+
+ // allocate target matrix block based on given size;
// need to allocate sparse as well since lock-free insert into target
long estnnz2 = (estnnz < 0) ? (long)nrow * ncol : estnnz;
return createOutputMatrixBlock(nrow, ncol, nrow, estnnz2, true, true);
}
-
+
private static class SplitOffsetInfos {
// offset & length info per split
private int[] offsetPerSplit = null;
@@ -222,8 +222,8 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
offsetPerSplit[split] = o;
}
}
-
- private static class CountRowsTask implements Callable<Object>
+
+ private static class CountRowsTask implements Callable<Object>
{
private InputSplit _split = null;
private TextInputFormat _informat = null;
@@ -246,14 +246,14 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
public int getRowCount() {
return _nrows;
}
-
+
public String getErrMsg() {
return _errMsg;
}
@Override
- public Object call()
- throws Exception
+ public Object call()
+ throws Exception
{
RecordReader<LongWritable, Text> reader = _informat.getRecordReader(_split, _job, Reporter.NULL);
LongWritable key = new LongWritable();
@@ -264,12 +264,12 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
while (reader.next(key, oneLine)) {
_nrows++;
}
- }
+ }
catch (Exception e) {
_rc = false;
_errMsg = "RecordReader error libsvm format. split: "+ _split.toString() + e.getMessage();
throw new IOException(_errMsg);
- }
+ }
finally {
IOUtilFunctions.closeSilently(reader);
}
@@ -277,8 +277,8 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
return null;
}
}
-
- private static class LIBSVMReadTask implements Callable<Object>
+
+ private static class LIBSVMReadTask implements Callable<Object>
{
private InputSplit _split = null;
private SplitOffsetInfos _splitoffsets = null;
@@ -287,14 +287,14 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
private MatrixBlock _dest = null;
private long _clen = -1;
private int _splitCount = 0;
-
+
private boolean _rc = true;
private Exception _exception = null;
private long _nnz;
-
+
public LIBSVMReadTask(InputSplit split, SplitOffsetInfos offsets,
TextInputFormat informat, JobConf job, MatrixBlock dest,
- long rlen, long clen, int splitCount)
+ long rlen, long clen, int splitCount)
{
_split = split;
_splitoffsets = offsets;
@@ -313,45 +313,47 @@ public class ReaderTextLIBSVMParallel extends MatrixReader
public Exception getException() {
return _exception;
}
-
+
public long getPartialNnz() {
return _nnz;
}
-
+
@Override
- public Object call()
- throws Exception
+ public Object call()
+ throws Exception
{
long lnnz = 0;
-
- try
+
+ try
{
RecordReader<LongWritable, Text> reader = _informat.getRecordReader(_split, _job, Reporter.NULL);
LongWritable key = new LongWritable();
Text value = new Text();
SparseRowVector vect = new SparseRowVector(1024);
-
+
int row = _splitoffsets.getOffsetPerSplit(_splitCount);
try {
while (reader.next(key, value)) { // foreach line
String rowStr = value.toString().trim();
- lnnz += ReaderTextLIBSVM.parseLibsvmRow(rowStr, vect, (int)_clen);
+ lnnz += ReaderTextLIBSVM
+ .parseLibsvmRow(rowStr, vect, (int) _clen, _props.getDelim(), _props.getIndexDelim());
_dest.appendRow(row, vect);
row++;
}
// sanity checks (number of rows)
if (row != (_splitoffsets.getOffsetPerSplit(_splitCount) + _splitoffsets.getLenghtPerSplit(_splitCount)) ) {
- throw new IOException("Incorrect number of rows ("+ row+ ") found in delimited file ("
- + (_splitoffsets.getOffsetPerSplit(_splitCount)
- + _splitoffsets.getLenghtPerSplit(_splitCount))+ "): " + value);
+ throw new IOException(
+ "Incorrect number of rows (" + row + ") found in delimited file (" + (_splitoffsets
+ .getOffsetPerSplit(_splitCount) + _splitoffsets
+ .getLenghtPerSplit(_splitCount)) + "): " + value);
}
- }
+ }
finally {
IOUtilFunctions.closeSilently(reader);
}
- }
+ }
catch (Exception ex) {
// central error handling (return code, message)
_rc = false;
diff --git a/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVM.java b/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVM.java
index f747b65..0bc7c2e 100644
--- a/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVM.java
+++ b/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVM.java
@@ -32,30 +32,32 @@ import org.apache.sysds.runtime.data.SparseBlock;
import org.apache.sysds.runtime.matrix.data.MatrixBlock;
import org.apache.sysds.runtime.util.HDFSTool;
-public class WriterTextLIBSVM extends MatrixWriter
-{
- public WriterTextLIBSVM() {
-
+public class WriterTextLIBSVM extends MatrixWriter {
+
+ protected static FileFormatPropertiesLIBSVM _props = null;
+
+ public WriterTextLIBSVM(FileFormatPropertiesLIBSVM _props) {
+ WriterTextLIBSVM._props = _props;
}
@Override
- public final void writeMatrixToHDFS(MatrixBlock src, String fname, long rlen, long clen, int blen, long nnz, boolean diag)
- throws IOException, DMLRuntimeException
+ public final void writeMatrixToHDFS(MatrixBlock src, String fname, long rlen, long clen, int blen, long nnz, boolean diag)
+ throws IOException, DMLRuntimeException
{
//validity check matrix dimensions
if( src.getNumRows() != rlen || src.getNumColumns() != clen )
throw new IOException("Matrix dimensions mismatch with metadata: "+src.getNumRows()+"x"+src.getNumColumns()+" vs "+rlen+"x"+clen+".");
if( rlen == 0 || clen == 0 )
throw new IOException("Write of matrices with zero rows or columns not supported ("+rlen+"x"+clen+").");
-
+
//prepare file access
JobConf job = new JobConf(ConfigurationManager.getCachedJobConf());
Path path = new Path( fname );
FileSystem fs = IOUtilFunctions.getFileSystem(path, job);
-
+
//if the file already exists on HDFS, remove it.
HDFSTool.deleteFileIfExistOnHDFS( fname );
-
+
//core write (sequential/parallel)
writeLIBSVMMatrixToHDFS(path, job, fs, src);
@@ -63,42 +65,43 @@ public class WriterTextLIBSVM extends MatrixWriter
}
@Override
- public final void writeEmptyMatrixToHDFS(String fname, long rlen, long clen, int blen)
- throws IOException, DMLRuntimeException
+ public final void writeEmptyMatrixToHDFS(String fname, long rlen, long clen, int blen)
+ throws IOException, DMLRuntimeException
{
-
+
}
- protected void writeLIBSVMMatrixToHDFS(Path path, JobConf job, FileSystem fs, MatrixBlock src)
- throws IOException
+ protected void writeLIBSVMMatrixToHDFS(Path path, JobConf job, FileSystem fs, MatrixBlock src)
+ throws IOException
{
//sequential write libsvm file
writeLIBSVMMatrixToFile(path, job, fs, src, 0, src.getNumRows());
}
-
+
protected static void writeLIBSVMMatrixToFile( Path path, JobConf job, FileSystem fs, MatrixBlock src, int rl, int rlen )
throws IOException
{
boolean sparse = src.isInSparseFormat();
int clen = src.getNumColumns();
-
+
//create buffered writer
BufferedWriter br = new BufferedWriter(new OutputStreamWriter(fs.create(path,true)));
-
+
try
{
StringBuilder sb = new StringBuilder();
-
+ _props = _props == null ? new FileFormatPropertiesLIBSVM() : _props;
+
// Write data lines
if( sparse ) //SPARSE
{
SparseBlock sblock = src.getSparseBlock();
for(int i=rl; i < rlen; i++) {
// append the class label as the 1st column
- double label = (sblock!=null) ?
+ double label = (sblock!=null) ?
sblock.get(i, clen-1) : 0;
sb.append(label);
-
+
if( sblock!=null && i<sblock.numRows() && !sblock.isEmpty(i) ) {
int pos = sblock.pos(i);
int alen = sblock.size(i);
@@ -107,7 +110,7 @@ public class WriterTextLIBSVM extends MatrixWriter
// append sparse row
for( int k=pos; k<pos+alen; k++ ) {
if( aix[k]!=clen-1 ) {
- sb.append(IOUtilFunctions.LIBSVM_DELIM);
+ sb.append(_props.getDelim());
appendIndexValLibsvm(sb, aix[k], avals[k]);
}
}
@@ -115,7 +118,7 @@ public class WriterTextLIBSVM extends MatrixWriter
// write the string row
sb.append('\n');
br.write( sb.toString() );
- sb.setLength(0);
+ sb.setLength(0);
}
}
else //DENSE
@@ -124,12 +127,12 @@ public class WriterTextLIBSVM extends MatrixWriter
// append the class label as the 1st column
double label = src.getValueDenseUnsafe(i, clen-1);
sb.append(label);
-
+
// append dense row
for( int j=0; j<clen-1; j++ ) {
double val = src.getValueDenseUnsafe(i, j);
if( val != 0 ) {
- sb.append(IOUtilFunctions.LIBSVM_DELIM);
+ sb.append(_props.getDelim());
appendIndexValLibsvm(sb, j, val);
}
}
@@ -145,10 +148,10 @@ public class WriterTextLIBSVM extends MatrixWriter
}
}
- // Return string in libsvm format (<index#>:<value#>)
+ // Return string in libsvm format (<index#>:<value#>)
protected static void appendIndexValLibsvm(StringBuilder sb, int index, double value) {
sb.append(index+1); // convert 0 based matrix index to 1 base libsvm index
- sb.append(IOUtilFunctions.LIBSVM_INDEX_DELIM);
+ sb.append(_props.getIndexDelim());
sb.append(value);
}
}
diff --git a/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVMParallel.java b/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVMParallel.java
index 1abe270..2053aca 100644
--- a/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVMParallel.java
+++ b/src/main/java/org/apache/sysds/runtime/io/WriterTextLIBSVMParallel.java
@@ -40,34 +40,34 @@ import org.apache.sysds.runtime.util.HDFSTool;
public class WriterTextLIBSVMParallel extends WriterTextLIBSVM
{
- public WriterTextLIBSVMParallel() {
- super();
+ public WriterTextLIBSVMParallel(FileFormatPropertiesLIBSVM _props) {
+ super(_props);
}
@Override
protected void writeLIBSVMMatrixToHDFS(Path path, JobConf job, FileSystem fs, MatrixBlock src)
- throws IOException
+ throws IOException
{
//estimate output size and number of output blocks (min 1)
int numPartFiles = (int)(OptimizerUtils.estimateSizeTextOutput(src.getNumRows(), src.getNumColumns(),
src.getNonZeros(), FileFormat.LIBSVM) / InfrastructureAnalyzer.getHDFSBlockSize());
numPartFiles = Math.max(numPartFiles, 1);
-
+
//determine degree of parallelism
int numThreads = OptimizerUtils.getParallelTextWriteParallelism();
numThreads = Math.min(numThreads, numPartFiles);
-
+
//fall back to sequential write if dop is 1 (e.g., <128MB) in order to create single file
if( numThreads <= 1 ) {
super.writeLIBSVMMatrixToHDFS(path, job, fs, src);
return;
}
-
+
//create directory for concurrent tasks
HDFSTool.createDirIfNotExistOnHDFS(path, DMLConfig.DEFAULT_SHARED_DIR_PERMISSION);
-
+
//create and execute tasks
- try
+ try
{
ExecutorService pool = CommonThreadPool.get(numThreads);
ArrayList<WriteLIBSVMTask> tasks = new ArrayList<>();
@@ -81,31 +81,31 @@ public class WriterTextLIBSVMParallel extends WriterTextLIBSVM
//wait until all tasks have been executed
List<Future<Object>> rt = pool.invokeAll(tasks);
pool.shutdown();
-
- //check for exceptions
+
+ //check for exceptions
for( Future<Object> task : rt )
task.get();
-
+
// delete crc files if written to local file system
if (fs instanceof LocalFileSystem) {
- for(int i=0; i<numThreads & i*blklen<rlen; i++)
+ for(int i=0; i<numThreads & i*blklen<rlen; i++)
IOUtilFunctions.deleteCrcFilesFromLocalFileSystem(fs,
new Path(path, IOUtilFunctions.getPartFileName(i)));
}
- }
+ }
catch (Exception e) {
throw new IOException("Failed parallel write of libsvm output.", e);
}
}
- private class WriteLIBSVMTask implements Callable<Object>
+ private class WriteLIBSVMTask implements Callable<Object>
{
private final JobConf _job;
private final FileSystem _fs;
private final MatrixBlock _src;
private final Path _path;
private final int _rl, _ru;
-
+
public WriteLIBSVMTask(Path path, JobConf job, FileSystem fs, MatrixBlock src, int rl, int ru) {
_path = path;
_job = job;
diff --git a/src/test/java/org/apache/sysds/test/functions/data/misc/NoRenameTest.java b/src/test/java/org/apache/sysds/test/functions/data/misc/NoRenameTest.java
index 781edfa..be46bd4 100644
--- a/src/test/java/org/apache/sysds/test/functions/data/misc/NoRenameTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/data/misc/NoRenameTest.java
@@ -187,16 +187,16 @@ public class NoRenameTest extends AutomatedTestBase
runRenameTest("mm", true, ExecMode.SPARK);
}
-// @Test
-// public void testTextlibsvmDenseSpark() {
-// runRenameTest("libsvm", false, ExecMode.SPARK);
-// }
-//
-// @Test
-// public void testTextlibsvmSparseSpark() {
-// runRenameTest("libsvm", true, ExecMode.SPARK);
-// }
-//
+ @Test
+ public void testTextlibsvmDenseSpark() {
+ runRenameTest("libsvm", false, ExecMode.SPARK);
+ }
+
+ @Test
+ public void testTextlibsvmSparseSpark() {
+ runRenameTest("libsvm", true, ExecMode.SPARK);
+ }
+
@Test
public void testBinaryDenseSpark() {
runRenameTest("binary", false, ExecMode.SPARK);
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/LIBSVMConfig.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/LIBSVMConfig.java
new file mode 100644
index 0000000..a818507
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/LIBSVMConfig.java
@@ -0,0 +1,77 @@
+/*
+ * 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.io.libsvm;
+
+public class LIBSVMConfig {
+
+ private String inSep;
+ private String inIndSep;
+ private int colCount;
+ private String outSep;
+ private String outIndSep;
+
+ public LIBSVMConfig(String inSep, String inIndSep, int colCount, String outSep, String outIndSep) {
+ this.inSep = inSep;
+ this.inIndSep = inIndSep;
+ this.colCount = colCount;
+ this.outSep = outSep;
+ this.outIndSep = outIndSep;
+ }
+
+ public String getInSep() {
+ return inSep;
+ }
+
+ public void setInSep(String inSep) {
+ this.inSep = inSep;
+ }
+
+ public String getInIndSep() {
+ return inIndSep;
+ }
+
+ public void setInIndSep(String inIndSep) {
+ this.inIndSep = inIndSep;
+ }
+
+ public int getColCount() {
+ return colCount;
+ }
+
+ public void setColCount(int colCount) {
+ this.colCount = colCount;
+ }
+
+ public String getOutSep() {
+ return outSep;
+ }
+
+ public void setOutSep(String outSep) {
+ this.outSep = outSep;
+ }
+
+ public String getOutIndSep() {
+ return outIndSep;
+ }
+
+ public void setOutIndSep(String outIndSep) {
+ this.outIndSep = outIndSep;
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest.java
index f628a8d..f679aa7 100644
--- a/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest.java
@@ -19,63 +19,81 @@
package org.apache.sysds.test.functions.io.libsvm;
-import org.junit.Test;
+import org.apache.sysds.api.DMLScript;
import org.apache.sysds.common.Types.ExecMode;
import org.apache.sysds.conf.CompilerConfig;
-import org.apache.sysds.test.AutomatedTestBase;
import org.apache.sysds.test.TestConfiguration;
import org.apache.sysds.test.TestUtils;
+import org.junit.Test;
+
+public abstract class ReadLIBSVMTest extends ReadLIBSVMTestBase {
+
+ protected abstract int getId();
+
+ protected abstract LIBSVMConfig getLIBSVMConfig();
-public class ReadLIBSVMTest extends AutomatedTestBase
-{
- private final static String TEST_NAME = "ReadLIBSVMTest";
- private final static String TEST_DIR = "functions/io/libsvm/";
- private final static String TEST_CLASS_DIR = TEST_DIR + ReadLIBSVMTest.class.getSimpleName() + "/";
-
- @Override
- public void setUp()
- {
- TestUtils.clearAssertionInformation();
- addTestConfiguration(TEST_NAME,
- new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[] { "Rout" }) );
+ protected String getInputLIBSVMFileName() {
+ return "transfusion_" + getId() + ".libsvm";
}
-
- @Test
- public void testlibsvm1_Seq_CP() {
- runlibsvmTest(1, ExecMode.SINGLE_NODE, false);
+
+ private final static double eps = 1e-9;
+
+ @Test public void testlibsvm1_Seq_CP() {
+ runlibsvmTest(getId(), ExecMode.SINGLE_NODE, false, getLIBSVMConfig());
}
- @Test
- public void testlibsvm2_Seq_CP() {
- runlibsvmTest(2, ExecMode.SINGLE_NODE, false);
+ @Test public void testlibsvm2_Pllel_CP() {
+ runlibsvmTest(getId(), ExecMode.SINGLE_NODE, true, getLIBSVMConfig());
}
- @Test
- public void testlibsvm2_Pllel_CP() {
- runlibsvmTest(2, ExecMode.SINGLE_NODE, true);
+ @Test public void testlibsvm3_SP() {
+ runlibsvmTest(getId(), ExecMode.SPARK, false, getLIBSVMConfig());
}
-
- private void runlibsvmTest (int testNumber, ExecMode platform, boolean parallel)
- {
+
+ protected void runlibsvmTest(int testNumber, ExecMode platform, boolean parallel, LIBSVMConfig libsvmConfig) {
ExecMode oldPlatform = rtplatform;
rtplatform = platform;
+
+ boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+ if(rtplatform == ExecMode.SPARK)
+ DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+
boolean oldpar = CompilerConfig.FLAG_PARREADWRITE_TEXT;
-
- try
- {
+
+ try {
CompilerConfig.FLAG_PARREADWRITE_TEXT = parallel;
- loadTestConfiguration(getTestConfiguration(TEST_NAME));
+
+ TestConfiguration config = getTestConfiguration(getTestName());
+ loadTestConfiguration(config);
+
String HOME = SCRIPT_DIR + TEST_DIR;
- String inputMatrix = HOME + "test" + testNumber+".libsvm";
+ String inputMatrix = HOME + INPUT_DIR + getInputLIBSVMFileName();
String dmlOutput = output("dml.scalar");
-
- fullDMLScriptName = HOME + TEST_NAME + "_" + testNumber + ".dml";
- programArgs = new String[]{"-explain", "hops", "-args", inputMatrix, dmlOutput};
+ String rOutput = output("R.scalar");
+ String sep = libsvmConfig.getOutSep();
+ String indSep = libsvmConfig.getOutIndSep();
+
+ fullDMLScriptName = HOME + getTestName() + "_" + testNumber + ".dml";
+ programArgs = new String[] {"-explain", "hops", "-args", inputMatrix, dmlOutput};
runTest(true, false, null, -1);
+
+ fullRScriptName = HOME + "libsvm_verify.R";
+
+ if(sep.equals(" ")) {
+ sep = "NULL";
+ }
+ rCmd = "Rscript" + " " + fullRScriptName + " " + inputMatrix + " " + libsvmConfig
+ .getColCount() + " " + sep + " " + indSep + " " + rOutput;
+ runRScript(true);
+
+ double dmlScalar = TestUtils.readDMLScalar(dmlOutput);
+ double rScalar = TestUtils.readRScalar(rOutput);
+ TestUtils.compareScalars(dmlScalar, rScalar, eps);
}
finally {
rtplatform = oldPlatform;
CompilerConfig.FLAG_PARREADWRITE_TEXT = oldpar;
+ DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
}
}
}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest1.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest1.java
new file mode 100644
index 0000000..eeabe67
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest1.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.functions.io.libsvm;
+
+public class ReadLIBSVMTest1 extends ReadLIBSVMTest {
+
+ private final static String TEST_NAME = "ReadLIBSVMTest";
+ private final static String TEST_CLASS_DIR = TEST_DIR + ReadLIBSVMTest1.class.getSimpleName() + "/";
+
+ protected String getTestName() {
+ return TEST_NAME;
+ }
+
+ protected String getTestClassDir() {
+ return TEST_CLASS_DIR;
+ }
+
+ protected int getId() {
+ return 1;
+ }
+
+ protected LIBSVMConfig getLIBSVMConfig() {
+ return new LIBSVMConfig(" ", ":", 2, " ", ":");
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest2.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest2.java
new file mode 100644
index 0000000..3b377cb
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest2.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.functions.io.libsvm;
+
+public class ReadLIBSVMTest2 extends ReadLIBSVMTest {
+
+ private final static String TEST_NAME = "ReadLIBSVMTest";
+ private final static String TEST_CLASS_DIR = TEST_DIR + ReadLIBSVMTest2.class.getSimpleName() + "/";
+
+ protected String getTestName() {
+ return TEST_NAME;
+ }
+
+ protected String getTestClassDir() {
+ return TEST_CLASS_DIR;
+ }
+
+ protected int getId() {
+ return 2;
+ }
+
+ protected LIBSVMConfig getLIBSVMConfig() {
+ return new LIBSVMConfig(" ", ":", 124, " ", ":");
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest3.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest3.java
new file mode 100644
index 0000000..b3ca405
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTest3.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.functions.io.libsvm;
+
+public class ReadLIBSVMTest3 extends ReadLIBSVMTest {
+
+ private final static String TEST_NAME = "ReadLIBSVMTest";
+ private final static String TEST_CLASS_DIR = TEST_DIR + ReadLIBSVMTest3.class.getSimpleName() + "/";
+
+ protected String getTestName() {
+ return TEST_NAME;
+ }
+
+ protected String getTestClassDir() {
+ return TEST_CLASS_DIR;
+ }
+
+ protected int getId() {
+ return 3;
+ }
+
+ protected LIBSVMConfig getLIBSVMConfig() {
+ return new LIBSVMConfig(" ", ":", 2, " ", ":");
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTestBase.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTestBase.java
new file mode 100644
index 0000000..05ac6ec
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/ReadLIBSVMTestBase.java
@@ -0,0 +1,38 @@
+/*
+ * 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.io.libsvm;
+
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+
+public abstract class ReadLIBSVMTestBase extends AutomatedTestBase {
+
+ protected final static String TEST_DIR = "functions/io/libsvm/";
+ protected final static String TEST_CLASS_DIR = TEST_DIR + ReadLIBSVMTest.class.getSimpleName() + "/";
+
+ protected abstract String getTestClassDir();
+
+ protected abstract String getTestName();
+
+ @Override public void setUp() {
+ addTestConfiguration(getTestName(),
+ new TestConfiguration(getTestClassDir(), getTestName(), new String[] {"Rout"}));
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest.java
index 026ca85..eb9955b 100644
--- a/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest.java
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest.java
@@ -19,56 +19,108 @@
package org.apache.sysds.test.functions.io.libsvm;
+import org.apache.sysds.api.DMLScript;
+import org.apache.sysds.conf.CompilerConfig;
+import org.apache.sysds.test.TestUtils;
import org.junit.Test;
import org.apache.sysds.common.Types.ExecMode;
-import org.apache.sysds.test.AutomatedTestBase;
import org.apache.sysds.test.TestConfiguration;
-public class WriteLIBSVMTest extends AutomatedTestBase
-{
- private final static String TEST_NAME = "WriteLIBSVMTest";
- private final static String TEST_DIR = "functions/io/libsvm/";
- private final static String TEST_CLASS_DIR = TEST_DIR + WriteLIBSVMTest.class.getSimpleName() + "/";
-
- @Override
- public void setUp()
- {
- addTestConfiguration(TEST_NAME,
- new TestConfiguration(TEST_CLASS_DIR, TEST_NAME, new String[] { "Rout" }) );
+public abstract class WriteLIBSVMTest extends WriteLIBSVMTestBase {
+
+ protected abstract int getId();
+
+ protected abstract LIBSVMConfig getLIBSVMConfig();
+
+ protected String getInputLIBSVMFileName() {
+ return "transfusion_W" + getId() + ".libsvm";
}
-
- @Test
- public void testLIBSVM1_CP() {
- runWriteLIBSVMTest(ExecMode.SINGLE_NODE, 1, 1, false);
+
+ private final static double eps = 1e-9;
+
+ @Test public void testlibsvm1_Seq_CP() {
+ runWriteLIBSVMTest(getId(), ExecMode.SINGLE_NODE, false, getLIBSVMConfig(), false);
+ }
+
+ @Test public void testlibsvm2_Seq_CP() {
+ runWriteLIBSVMTest(getId(), ExecMode.SINGLE_NODE, false, getLIBSVMConfig(), true);
}
- @Test
- public void testLIBSVM2_CP() {
- runWriteLIBSVMTest(ExecMode.SINGLE_NODE, 1, 2, true);
+ @Test public void testlibsvm1_Pllel_CP() {
+ runWriteLIBSVMTest(getId(), ExecMode.SINGLE_NODE, true, getLIBSVMConfig(), true);
}
- @Test
- public void testLIBSVM3_CP() {
- runWriteLIBSVMTest(ExecMode.SINGLE_NODE, 2, 3, true);
+ @Test public void testlibsvm2_Pllel_CP() {
+ runWriteLIBSVMTest(getId(), ExecMode.SINGLE_NODE, true, getLIBSVMConfig(), false);
}
-
- private void runWriteLIBSVMTest(ExecMode platform, int tno, int mno, boolean sparse) {
+
+ @Test public void testlibsvm1_SP() {
+ runWriteLIBSVMTest(getId(), ExecMode.SPARK, false, getLIBSVMConfig(), true);
+ }
+
+ @Test public void testlibsvm2_SP() {
+ runWriteLIBSVMTest(getId(), ExecMode.SPARK, false, getLIBSVMConfig(), false);
+ }
+
+ protected void runWriteLIBSVMTest(int testNumber, ExecMode platform, boolean parallel, LIBSVMConfig libsvmConfig,
+ boolean sparse) {
+
ExecMode oldPlatform = rtplatform;
rtplatform = platform;
-
- TestConfiguration config = getTestConfiguration(TEST_NAME);
- loadTestConfiguration(config);
-
- String HOME = SCRIPT_DIR + TEST_DIR;
- String inputMatrixName = HOME + "test" + mno +"w.libsvm";
- String dmlOutput = output("dml.scalar");
- String libsvmOutputName = output("libsvm_dml.data");
-
- fullDMLScriptName = HOME + TEST_NAME + "_" + tno + ".dml";
- programArgs = new String[]{"-explain", "-args", inputMatrixName, dmlOutput, libsvmOutputName,
- Boolean.toString(sparse) };
-
- runTest(true, false, null, -1);
- rtplatform = oldPlatform;
+
+ boolean sparkConfigOld = DMLScript.USE_LOCAL_SPARK_CONFIG;
+ if(rtplatform == ExecMode.SPARK)
+ DMLScript.USE_LOCAL_SPARK_CONFIG = true;
+
+ boolean oldpar = CompilerConfig.FLAG_PARREADWRITE_TEXT;
+
+ try {
+
+ CompilerConfig.FLAG_PARREADWRITE_TEXT = parallel;
+
+ TestConfiguration config = getTestConfiguration(getTestName());
+ loadTestConfiguration(config);
+
+ String HOME = SCRIPT_DIR + TEST_DIR;
+ String inputMatrixName = HOME + INPUT_DIR + getInputLIBSVMFileName();
+ String dmlOutput = output("dml.scalar");
+ String rOutput = output("R.scalar");
+ String libsvmOutputName = output("libsvm_write" + testNumber + ".data");
+ String sep = libsvmConfig.getOutSep();
+ String indSep = libsvmConfig.getOutIndSep();
+
+ fullDMLScriptName = HOME + getTestName() + "_" + testNumber + ".dml";
+ programArgs = new String[] {"-args", inputMatrixName, dmlOutput, libsvmOutputName, sep, indSep,
+ Boolean.toString(sparse)};
+
+ runTest(true, false, null, -1);
+
+ fullRScriptName = HOME + "libsvm_verify.R";
+
+ // the R scripts can't read HFDS file systems, so for the spark test
+ // we just read the original file stream and calc the sum of the cells.
+ // Then, compare the result with scalar dml
+ if(platform == ExecMode.SPARK) {
+ libsvmOutputName = inputMatrixName;
+ sep = libsvmConfig.getInSep();
+ indSep = libsvmConfig.getInIndSep();
+ }
+
+ if(sep.equals(" ")) {
+ sep = "NULL";
+ }
+ rCmd = "Rscript" + " " + fullRScriptName + " " + libsvmOutputName + " " + libsvmConfig
+ .getColCount() + " " + sep + " " + indSep + " " + rOutput;
+ runRScript(true);
+
+ double dmlScalar = TestUtils.readDMLScalar(dmlOutput);
+ double rScalar = TestUtils.readRScalar(rOutput);
+ TestUtils.compareScalars(dmlScalar, rScalar, eps);
+ }
+ finally {
+ rtplatform = oldPlatform;
+ CompilerConfig.FLAG_PARREADWRITE_TEXT = oldpar;
+ DMLScript.USE_LOCAL_SPARK_CONFIG = sparkConfigOld;
+ }
}
}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest1.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest1.java
new file mode 100644
index 0000000..3fa6d47
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest1.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sysds.test.functions.io.libsvm;
+
+public class WriteLIBSVMTest1 extends WriteLIBSVMTest {
+
+ private final static String TEST_NAME = "WriteLIBSVMTest";
+ public final static String TEST_CLASS_DIR = TEST_DIR + WriteLIBSVMTest1.class.getSimpleName() + "/";
+
+ protected String getTestName() {
+ return TEST_NAME;
+ }
+
+ protected String getTestClassDir() {
+ return TEST_CLASS_DIR;
+ }
+
+ protected int getId() {
+ return 1;
+ }
+
+ protected LIBSVMConfig getLIBSVMConfig() {
+ return new LIBSVMConfig(" ", ":", 3, " ", ":");
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest2.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest2.java
new file mode 100644
index 0000000..744dbd3
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTest2.java
@@ -0,0 +1,43 @@
+/*
+ * 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.io.libsvm;
+
+public class WriteLIBSVMTest2 extends WriteLIBSVMTest {
+
+ private final static String TEST_NAME = "WriteLIBSVMTest";
+ private final static String TEST_CLASS_DIR = TEST_DIR + WriteLIBSVMTest2.class.getSimpleName() + "/";
+
+ protected String getTestName() {
+ return TEST_NAME;
+ }
+
+ protected String getTestClassDir() {
+ return TEST_CLASS_DIR;
+ }
+
+ protected int getId() {
+ return 2;
+ }
+
+ protected LIBSVMConfig getLIBSVMConfig() {
+
+ return new LIBSVMConfig(" ", ":", 124, ",", ":");
+ }
+}
diff --git a/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTestBase.java b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTestBase.java
new file mode 100644
index 0000000..2876650
--- /dev/null
+++ b/src/test/java/org/apache/sysds/test/functions/io/libsvm/WriteLIBSVMTestBase.java
@@ -0,0 +1,40 @@
+/*
+ * 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.io.libsvm;
+
+import org.apache.sysds.test.AutomatedTestBase;
+import org.apache.sysds.test.TestConfiguration;
+import org.apache.sysds.test.TestUtils;
+
+public abstract class WriteLIBSVMTestBase extends AutomatedTestBase {
+
+ protected final static String TEST_DIR = "functions/io/libsvm/";
+ protected final static String TEST_CLASS_DIR = TEST_DIR + WriteLIBSVMTest.class.getSimpleName() + "/";
+
+ protected abstract String getTestClassDir();
+
+ protected abstract String getTestName();
+
+ @Override public void setUp() {
+ TestUtils.clearAssertionInformation();
+ addTestConfiguration(getTestName(),
+ new TestConfiguration(getTestClassDir(), getTestName(), new String[] {"Rout"}));
+ }
+}
diff --git a/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml b/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml
index b474397..81f48c7 100644
--- a/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml
+++ b/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml
@@ -22,7 +22,7 @@
# DML script that tests read libsvm with .mtd file
A = read($1);
-print(toString(A));
+A = A + 1;
x = sum(A);
write(x, $2);
diff --git a/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_2.dml b/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_2.dml
index c92dcdc..ed0f90c 100644
--- a/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_2.dml
+++ b/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_2.dml
@@ -21,7 +21,8 @@
# DML script that tests read libsvm
-A = read($1, format="libsvm", rows=1605, cols=124);
+A = read($1, format="libsvm", rows=1605, cols=124, sep=" ", indSep=":");
+A = A + 1;
x = sum(A);
write(x, $2);
diff --git a/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml b/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_3.dml
similarity index 94%
copy from src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml
copy to src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_3.dml
index b474397..c0d1f20 100644
--- a/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_1.dml
+++ b/src/test/scripts/functions/io/libsvm/ReadLIBSVMTest_3.dml
@@ -21,8 +21,8 @@
# DML script that tests read libsvm with .mtd file
-A = read($1);
-print(toString(A));
+A = read($1, format="libsvm", cols=2, sep=" ");
+A = A + 1;
x = sum(A);
write(x, $2);
diff --git a/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml b/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml
index ec8d1cc..0876d9a 100644
--- a/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml
+++ b/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml
@@ -21,10 +21,10 @@
# DML script that tests read/write libsvm
-A = read($1, format="libsvm", rows=7, cols=5);
+A = read($1);
+A = A + 0;
+write(A, $3, format="libsvm", sep=$4, indSep=$5, sparse=$6);
+A = read($3, format="libsvm", sep=$4, indSep=$5);
A = A + 1;
x = sum(A);
-write(x, $2);
-
-write(A, $3, format="libsvm", sparse=$4);
-
+write(x, $2);
\ No newline at end of file
diff --git a/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_2.dml b/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_2.dml
index f7b23f2..9e312de 100644
--- a/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_2.dml
+++ b/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_2.dml
@@ -21,11 +21,12 @@
# DML script that tests read/write libsvm
-A = read($1, format="libsvm", rows=1605, cols=124);
+A = read($1);
+A = A + 0;
+write(A, $3, format="libsvm", sep=$4, indSep=$5, sparse=$6);
+A = read($3, format="libsvm", sep=$4, indSep=$5);
A = A + 1;
x = sum(A);
write(x, $2);
-A = A - 1;
-write(A, $3, format="libsvm", sparse=$4);
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_1.libsvm b/src/test/scripts/functions/io/libsvm/in/transfusion_1.libsvm
new file mode 100644
index 0000000..378e5e9
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_1.libsvm
@@ -0,0 +1,6 @@
+1 1:0.25 2:1.
+1 1:0.4 2:0.5
+-1 1:0.5 2:0.15
+1 1:0.5 2:0.15
+1 1:0.5 2:0.15
+-1 1:0.5 2:0.15
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_1.libsvm.mtd b/src/test/scripts/functions/io/libsvm/in/transfusion_1.libsvm.mtd
new file mode 100644
index 0000000..7188bd8
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_1.libsvm.mtd
@@ -0,0 +1,11 @@
+{
+ "data_type": "matrix",
+ "value_type": "double",
+ "rows": 6,
+ "cols": 2,
+ "sep": " ",
+ "indSep": ":",
+ "format": "libsvm",
+ "author": "Saeed Fathollahzadeh",
+ "created": "2021-04-14 17:27:25 CEST"
+}
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_2.libsvm b/src/test/scripts/functions/io/libsvm/in/transfusion_2.libsvm
new file mode 100644
index 0000000..150ee74
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_2.libsvm
@@ -0,0 +1,1605 @@
+-1 3:1 11:1 14:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 27:1 35:1 40:1 57:1 63:1 69:1 73:1 74:1 76:1 81:1 103:1
+-1 4:1 6:1 15:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 47:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 41:1 47:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 48:1 64:1 71:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 11:1 15:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 16:1 30:1 35:1 41:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 29:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 11:1 18:1 20:1 37:1 42:1 59:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 18:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 18:1 20:1 37:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 107:1
+-1 1:1 6:1 14:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 44:1 51:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 10:1 18:1 22:1 36:1 42:1 56:1 64:1 67:1 73:1 74:1 76:1 80:1 100:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 27:1 35:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 10:1 18:1 22:1 36:1 43:1 51:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 7:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 15:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 15:1 21:1 35:1 45:1 62:1 68:1 72:1 74:1 77:1 80:1 93:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 55:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 5:1 6:1 16:1 20:1 37:1 41:1 55:1 64:1 68:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 25:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 42:1 50:1 64:1 70:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
++1 4:1 6:1 14:1 29:1 39:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 26:1 35:1 41:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 16:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 51:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 22:1 36:1 42:1 56:1 66:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 6:1 16:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 3:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 87:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 42:1 50:1 64:1 71:1 72:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 25:1 38:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 16:1 22:1 36:1 41:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 17:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 7:1 17:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 44:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 16:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 43:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 24:1 38:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 1:1 6:1 14:1 25:1 38:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 55:1 62:1 68:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 11:1 14:1 22:1 36:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 71:1 72:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 15:1 26:1 35:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 87:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 75:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 49:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 48:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 16:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 49:1 66:1 67:1 73:1 74:1 76:1 80:1 119:1
++1 4:1 6:1 14:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 10:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 48:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 42:1 56:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 20:1 37:1 42:1 55:1 65:1 71:1 72:1 74:1 76:1 80:1 101:1
+-1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 17:1 25:1 38:1 41:1 50:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 9:1 16:1 24:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 44:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 16:1 19:1 39:1 43:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 81:1 83:1
++1 3:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 10:1 17:1 19:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 15:1 29:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 10:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 43:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 14:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 68:1 73:1 74:1 76:1 81:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 33:1 35:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 15:1 29:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 9:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 9:1 18:1 21:1 35:1 42:1 47:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 11:1 18:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 17:1 20:1 37:1 43:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 95:1
+-1 3:1 6:1 18:1 23:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 77:1 80:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 9:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 59:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 9:1 14:1 22:1 36:1 46:1 50:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 43:1 55:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 10:1 15:1 29:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 34:1 35:1 42:1 49:1 65:1 67:1 72:1 74:1 76:1 79:1 103:1
+-1 1:1 18:1 22:1 36:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 10:1 17:1 21:1 35:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 16:1 22:1 36:1 42:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 11:1 15:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 17:1 25:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 88:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 40:1 50:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 14:1 24:1 38:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 45:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 100:1
++1 2:1 10:1 18:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 4:1 6:1 14:1 20:1 37:1 42:1 56:1 65:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 21:1 35:1 41:1 48:1 65:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 43:1 49:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 16:1 22:1 36:1 40:1 63:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 25:1 38:1 40:1 54:1 63:1 69:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 11:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 10:1 18:1 24:1 38:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 8:1 18:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 47:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 16:1 29:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 53:1 62:1 71:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 16:1 19:1 39:1 41:1 51:1 64:1 71:1 73:1 75:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 87:1
+-1 4:1 6:1 17:1 33:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 79:1 103:1
+-1 2:1 6:1 18:1 20:1 37:1 41:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 10:1 15:1 22:1 36:1 42:1 59:1 65:1 71:1 73:1 74:1 77:1 80:1 83:1
++1 3:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 11:1 15:1 19:1 39:1 42:1 52:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 4:1 6:1 14:1 25:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 6:1 16:1 20:1 37:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 9:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 16:1 20:1 37:1 40:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 16:1 20:1 37:1 42:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 49:1 63:1 71:1 73:1 74:1 76:1 82:1 85:1
+-1 5:1 6:1 14:1 29:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 14:1 22:1 36:1 43:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 18:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 53:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 11:1 15:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 8:1 16:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 19:1 39:1 43:1 51:1 64:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 68:1 72:1 74:1 76:1 80:1 102:1
+-1 5:1 6:1 14:1 22:1 36:1 44:1 49:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 42:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 16:1 28:1 35:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1
+-1 5:1 6:1 15:1 20:1 37:1 44:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 99:1
+-1 2:1 7:1 17:1 22:1 36:1 41:1 56:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 40:1 50:1 63:1 70:1 73:1 74:1 76:1 82:1
++1 5:1 6:1 15:1 24:1 38:1 40:1 51:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 9:1 14:1 25:1 38:1 42:1 52:1 65:1 68:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 10:1 15:1 19:1 39:1 42:1 52:1 64:1 71:1 73:1 75:1 76:1 81:1
++1 5:1 7:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 27:1 35:1 42:1 56:1 66:1 70:1 73:1 74:1 76:1 79:1 103:1
+-1 4:1 7:1 14:1 20:1 37:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 48:1 65:1 68:1 72:1 74:1 76:1 80:1 84:1
++1 4:1 8:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 41:1 49:1 66:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 8:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 10:1 17:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 31:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 79:1 103:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 10:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 7:1 18:1 30:1 35:1 45:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 4:1 6:1 18:1 30:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 22:1 36:1 41:1 57:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 15:1 25:1 38:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 10:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 17:1 29:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 26:1 35:1 43:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 119:1
++1 1:1 6:1 16:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 18:1 29:1 39:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 119:1
++1 3:1 11:1 14:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 79:1 83:1
+-1 5:1 6:1 15:1 33:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1
++1 4:1 6:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 8:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 43:1 57:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 7:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 88:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 7:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 41:1 48:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 58:1 66:1 71:1 72:1 74:1 76:1 78:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 54:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 50:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 10:1 16:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 17:1 19:1 39:1 43:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 41:1 54:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 8:1 15:1 20:1 37:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 11:1 18:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 17:1 20:1 37:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 10:1 15:1 20:1 37:1 43:1 59:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 31:1 35:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 32:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 7:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 18:1 22:1 36:1 44:1 51:1 66:1 68:1 72:1 74:1 76:1 82:1 93:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 49:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 15:1 27:1 35:1 44:1 55:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 11:1 17:1 23:1 39:1 42:1 52:1 64:1 68:1 73:1 75:1 76:1 82:1 90:1
+-1 3:1 7:1 15:1 20:1 37:1 40:1 48:1 63:1 69:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 14:1 27:1 35:1 45:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 99:1
+-1 3:1 17:1 25:1 38:1 42:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 33:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 78:1 103:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 47:1 64:1 71:1 73:1 74:1 76:1 80:1
+-1 2:1 6:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 17:1 20:1 37:1 41:1 51:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 18:1 21:1 35:1 41:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 27:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 43:1 51:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 9:1 14:1 20:1 37:1 41:1 47:1 66:1 69:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 19:1 39:1 42:1 55:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 49:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 82:1 88:1
+-1 1:1 16:1 25:1 38:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 2:1 7:1 17:1 19:1 39:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 11:1 18:1 19:1 39:1 41:1 59:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 47:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 7:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 18:1 29:1 39:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 107:1
+-1 3:1 6:1 15:1 22:1 36:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 21:1 35:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 119:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 41:1 55:1 64:1 68:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 16:1 33:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 1:1 6:1 18:1 27:1 35:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 9:1 15:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 17:1 19:1 39:1 43:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 24:1 38:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 14:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 25:1 38:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 17:1 23:1 39:1 41:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 18:1 31:1 35:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 23:1 39:1 43:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 7:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 26:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 9:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 22:1 36:1 40:1 50:1 63:1 68:1 73:1 74:1 76:1 82:1 93:1
+-1 5:1 6:1 17:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 2:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 50:1 62:1 68:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 18:1 19:1 39:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 8:1 15:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 58:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 25:1 38:1 42:1 51:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 27:1 35:1 44:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 16:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 14:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 10:1 18:1 28:1 35:1 42:1 49:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 7:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 42:1 52:1 66:1 67:1 72:1 74:1 77:1 80:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 46:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 25:1 38:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 16:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 41:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 100:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 5:1 7:1 16:1 22:1 36:1 42:1 56:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 10:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 9:1 16:1 23:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 2:1 7:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 40:1 55:1 61:1 69:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 14:1 23:1 39:1 40:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 7:1 17:1 20:1 37:1 41:1 56:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 14:1 29:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 32:1 39:1 41:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 77:1 78:1 83:1
++1 3:1 6:1 14:1 19:1 39:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 77:1 78:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 110:1
+-1 3:1 6:1 15:1 20:1 37:1 43:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 25:1 38:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 14:1 25:1 38:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 68:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 33:1 35:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 78:1 103:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 70:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 26:1 35:1 40:1 54:1 65:1 68:1 72:1 74:1 76:1 81:1 94:1
+-1 1:1 6:1 15:1 25:1 38:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 10:1 18:1 31:1 35:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 11:1 16:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 17:1 29:1 39:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 57:1 64:1 69:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 2:1 10:1 16:1 24:1 38:1 40:1 59:1 63:1 67:1 73:1 74:1 77:1 80:1 85:1
+-1 4:1 6:1 15:1 22:1 36:1 44:1 50:1 66:1 69:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 10:1 14:1 22:1 36:1 40:1 51:1 63:1 69:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 7:1 16:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 18:1 20:1 37:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 41:1 49:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 16:1 21:1 35:1 42:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 31:1 35:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 11:1 17:1 29:1 39:1 40:1 52:1 61:1 67:1 72:1 75:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 18:1 22:1 36:1 44:1 49:1 65:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 55:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 15:1 23:1 39:1 45:1 66:1 68:1 72:1 74:1 76:1 80:1 102:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 14:1 19:1 39:1 41:1 52:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 43:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 16:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 31:1 35:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 11:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 112:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 53:1 61:1 67:1 72:1 74:1 76:1 80:1 103:1
++1 5:1 7:1 14:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 85:1
+-1 2:1 10:1 18:1 19:1 39:1 42:1 59:1 62:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 54:1 65:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 2:1 6:1 14:1 20:1 37:1 40:1 49:1 61:1 69:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 66:1 71:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 31:1 35:1 40:1 48:1 61:1 70:1 72:1 74:1 76:1 82:1 114:1
+-1 4:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 54:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1
+-1 3:1 8:1 15:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 16:1 24:1 38:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 21:1 35:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 56:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 27:1 35:1 42:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 9:1 14:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 82:1 105:1
+-1 2:1 7:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 15:1 34:1 35:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 107:1
+-1 2:1 6:1 15:1 26:1 35:1 40:1 56:1 63:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 8:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 10:1 17:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 68:1 73:1 74:1 76:1 80:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 86:1
+-1 5:1 7:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 48:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 8:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 9:1 17:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 14:1 25:1 38:1 41:1 47:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 17:1 19:1 39:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 11:1 15:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 18:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 21:1 35:1 42:1 49:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 14:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 10:1 17:1 22:1 36:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 79:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 41:1 54:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 48:1 64:1 71:1 72:1 74:1 76:1 81:1 83:1
++1 5:1 9:1 16:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 43:1 57:1 65:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 15:1 22:1 36:1 43:1 48:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 31:1 35:1 42:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 29:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 10:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 2:1 14:1 20:1 37:1 43:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 21:1 35:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 27:1 35:1 41:1 53:1 65:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 7:1 14:1 22:1 36:1 41:1 56:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 58:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 3:1 11:1 14:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 31:1 35:1 42:1 50:1 66:1 67:1 72:1 74:1 76:1 79:1
+-1 5:1 10:1 16:1 33:1 35:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 45:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 14:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 33:1 35:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 112:1
+-1 3:1 6:1 17:1 19:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 10:1 14:1 20:1 37:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 27:1 35:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 29:1 39:1 42:1 51:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 15:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 85:1
+-1 5:1 6:1 17:1 33:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 4:1 10:1 15:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 52:1 62:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 49:1 66:1 71:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 50:1 65:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 24:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 10:1 16:1 26:1 35:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 118:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 43:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 7:1 15:1 22:1 36:1 41:1 51:1 64:1 67:1 72:1 75:1 76:1 79:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 7:1 18:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 18:1 20:1 37:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 52:1 64:1 71:1 72:1 75:1 76:1 82:1 83:1
++1 4:1 6:1 16:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 20:1 37:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 18:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 8:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 17:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 43:1 49:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 15:1 22:1 36:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 16:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 42:1 56:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 15:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 20:1 37:1 40:1 56:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 16:1 22:1 36:1 42:1 52:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 44:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 11:1 16:1 29:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 33:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 2:1 18:1 22:1 36:1 42:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 40:1 53:1 63:1 71:1 73:1 74:1 76:1 80:1
++1 5:1 6:1 14:1 19:1 39:1 42:1 49:1 64:1 68:1 72:1 74:1 76:1 78:1 91:1
+-1 5:1 6:1 16:1 34:1 35:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 7:1 18:1 22:1 36:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 17:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 62:1 69:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 3:1 7:1 16:1 19:1 39:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 28:1 35:1 42:1 57:1 64:1 68:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 7:1 16:1 20:1 37:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 24:1 38:1 42:1 52:1 64:1 68:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 29:1 39:1 43:1 51:1 64:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 11:1 16:1 20:1 37:1 42:1 49:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 23:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 7:1 14:1 24:1 38:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 42:1 54:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 42:1 54:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 9:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 11:1 18:1 19:1 39:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 14:1 30:1 35:1 42:1 49:1 65:1 67:1 72:1 74:1 76:1 78:1 119:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 9:1 15:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 14:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 58:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 50:1 64:1 71:1 73:1 74:1 76:1 80:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 48:1 66:1 67:1 72:1 74:1 76:1 80:1 86:1
++1 4:1 10:1 17:1 19:1 39:1 41:1 59:1 64:1 71:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 42:1 50:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 16:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 29:1 39:1 42:1 51:1 64:1 70:1 73:1 74:1 76:1 78:1 95:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 53:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 28:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 23:1 39:1 42:1 52:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 31:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 8:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 2:1 6:1 14:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 44:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 77:1 78:1 83:1
++1 4:1 9:1 17:1 20:1 37:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 17:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 18:1 27:1 35:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 26:1 35:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 26:1 35:1 43:1 53:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 3:1 6:1 16:1 20:1 37:1 40:1 54:1 63:1 68:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 16:1 30:1 35:1 40:1 49:1 65:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 50:1 62:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 16:1 24:1 38:1 42:1 65:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 8:1 16:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 14:1 23:1 39:1 45:1 52:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 18:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 10:1 18:1 19:1 39:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 9:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 18:1 20:1 37:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 18:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 17:1 25:1 38:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 13:1 17:1 31:1 35:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 77:1 79:1 83:1
+-1 3:1 10:1 15:1 25:1 38:1 42:1 51:1 66:1 69:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 25:1 38:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 8:1 16:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 44:1 58:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 31:1 35:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 25:1 38:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 52:1 64:1 67:1 72:1 74:1 77:1 81:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 51:1 62:1 68:1 72:1 74:1 76:1 82:1 110:1
++1 4:1 7:1 18:1 19:1 39:1 41:1 49:1 66:1 68:1 72:1 74:1 76:1 82:1 93:1
+-1 1:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 10:1 15:1 19:1 39:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 6:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 8:1 18:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 16:1 31:1 35:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 95:1
++1 3:1 7:1 17:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 40:1 58:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 17:1 29:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 17:1 25:1 38:1 42:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 18:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 31:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 18:1 26:1 35:1 43:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 9:1 17:1 25:1 38:1 41:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 18:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 77:1 80:1 83:1
++1 3:1 6:1 16:1 24:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 9:1 14:1 20:1 37:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 25:1 38:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 2:1 6:1 17:1 22:1 36:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 49:1 62:1 69:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 8:1 15:1 20:1 37:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 30:1 35:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 16:1 23:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 15:1 25:1 38:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 17:1 20:1 37:1 41:1 57:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 8:1 17:1 22:1 36:1 40:1 50:1 63:1 71:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 5:1 15:1 22:1 36:1 44:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 40:1 55:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 8:1 14:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 40:1 55:1 62:1 69:1 72:1 75:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 27:1 35:1 45:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 114:1
++1 1:1 6:1 15:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 100:1
++1 4:1 10:1 15:1 29:1 39:1 40:1 52:1 63:1 71:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 15:1 21:1 35:1 42:1 53:1 65:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 24:1 38:1 41:1 55:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 14:1 31:1 35:1 44:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 11:1 17:1 32:1 39:1 43:1 52:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 1:1 7:1 18:1 26:1 35:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 103:1
++1 4:1 9:1 18:1 20:1 37:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 19:1 39:1 42:1 55:1 62:1 68:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 24:1 38:1 40:1 50:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 4:1 17:1 25:1 38:1 41:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 41:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1
+-1 5:1 6:1 15:1 20:1 37:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 29:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 109:1
++1 3:1 6:1 14:1 19:1 39:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 41:1 55:1 65:1 69:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 27:1 35:1 44:1 49:1 66:1 69:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 15:1 19:1 39:1 42:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 16:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 9:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 10:1 15:1 20:1 37:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 100:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 51:1 64:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 18:1 19:1 39:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 19:1 39:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 17:1 29:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 16:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 5:1 7:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 24:1 38:1 42:1 49:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 26:1 35:1 42:1 48:1 66:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 16:1 29:1 39:1 42:1 55:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 42:1 51:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 21:1 35:1 42:1 49:1 62:1 71:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 10:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 15:1 22:1 36:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 15:1 26:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 26:1 35:1 40:1 54:1 63:1 71:1 73:1 75:1 76:1 80:1 83:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 25:1 38:1 40:1 47:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 3:1 11:1 17:1 20:1 37:1 43:1 51:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 79:1 83:1
++1 5:1 6:1 14:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 31:1 35:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 17:1 20:1 37:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 78:1 87:1
+-1 5:1 6:1 16:1 31:1 35:1 44:1 56:1 66:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 87:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 42:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 100:1
+-1 5:1 6:1 18:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 14:1 27:1 35:1 40:1 50:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 21:1 35:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 9:1 15:1 19:1 39:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 15:1 20:1 37:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 16:1 19:1 39:1 42:1 52:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 41:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 29:1 39:1 41:1 59:1 64:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 43:1 49:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 33:1 35:1 42:1 53:1 65:1 67:1 73:1 74:1 76:1 78:1 97:1
+-1 4:1 6:1 16:1 22:1 36:1 43:1 52:1 66:1 67:1 72:1 74:1 77:1 80:1 86:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 27:1 35:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 78:1 95:1
+-1 3:1 6:1 17:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 44:1 51:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 3:1 6:1 17:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 26:1 35:1 40:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 10:1 18:1 20:1 37:1 42:1 51:1 66:1 71:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 43:1 50:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 41:1 53:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 42:1 48:1 65:1 68:1 73:1 74:1 76:1 80:1 84:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 54:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 29:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 82:1 99:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 53:1 62:1 68:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 11:1 15:1 22:1 36:1 42:1 57:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 55:1 64:1 68:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 18:1 26:1 35:1 41:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 18:1 25:1 38:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 24:1 38:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 31:1 35:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 41:1 55:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 18:1 33:1 35:1 41:1 53:1 66:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 3:1 6:1 14:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 18:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 9:1 18:1 20:1 37:1 40:1 47:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
++1 5:1 7:1 16:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 51:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 8:1 15:1 29:1 39:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 9:1 14:1 19:1 39:1 45:1 51:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 21:1 35:1 45:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 18:1 29:1 39:1 41:1 51:1 64:1 71:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 16:1 25:1 38:1 42:1 48:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 24:1 38:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 78:1 113:1
+-1 4:1 6:1 17:1 33:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
++1 3:1 6:1 16:1 19:1 39:1 40:1 50:1 61:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 4:1 11:1 15:1 22:1 36:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 25:1 38:1 43:1 52:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 45:1 57:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 6:1 17:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 10:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 50:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 55:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 44:1 49:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 8:1 16:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 11:1 18:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 16:1 19:1 39:1 42:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 10:1 15:1 22:1 36:1 41:1 57:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 16:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 7:1 18:1 20:1 37:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 16:1 22:1 36:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 10:1 14:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 10:1 15:1 22:1 36:1 40:1 55:1 61:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 11:1 17:1 19:1 39:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 25:1 38:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 16:1 33:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 87:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 1:1 6:1 14:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 77:1 80:1 83:1
++1 2:1 7:1 14:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
++1 5:1 6:1 17:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 26:1 35:1 40:1 58:1 61:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 21:1 35:1 40:1 53:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 3:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 55:1 66:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 16:1 31:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 7:1 16:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 7:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 57:1 64:1 71:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 10:1 16:1 20:1 37:1 45:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 88:1
+-1 1:1 16:1 20:1 37:1 42:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 14:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 10:1 18:1 25:1 38:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 31:1 35:1 40:1 50:1 61:1 68:1 72:1 74:1 76:1 78:1 91:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 54:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 49:1 63:1 68:1 73:1 74:1 76:1 80:1 94:1
+-1 4:1 6:1 18:1 22:1 36:1 41:1 50:1 66:1 71:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 11:1 16:1 32:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 9:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 33:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 4:1 10:1 14:1 27:1 35:1 41:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 24:1 38:1 41:1 51:1 64:1 71:1 72:1 74:1 76:1 80:1 101:1
+-1 5:1 7:1 16:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 48:1 63:1 68:1 73:1 74:1 76:1 80:1 84:1
+-1 3:1 9:1 14:1 22:1 36:1 42:1 55:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 43:1 50:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 18:1 21:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 103:1
++1 4:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 43:1 50:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 42:1 52:1 66:1 70:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 49:1 64:1 68:1 72:1 74:1 76:1 79:1 117:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 53:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 14:1 20:1 37:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 20:1 37:1 45:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 11:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 18:1 19:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 45:1 55:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 10:1 15:1 24:1 38:1 41:1 49:1 64:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 55:1 65:1 67:1 73:1 74:1 76:1 78:1 115:1
+-1 5:1 6:1 18:1 31:1 35:1 44:1 54:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 7:1 17:1 19:1 39:1 41:1 50:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 119:1
+-1 4:1 6:1 17:1 20:1 37:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 24:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 18:1 24:1 38:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 24:1 38:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 10:1 14:1 19:1 39:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 42:1 48:1 62:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 25:1 38:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 10:1 17:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 9:1 18:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 14:1 31:1 35:1 40:1 63:1 69:1 73:1 75:1 76:1 78:1 83:1
+-1 5:1 15:1 20:1 37:1 41:1 64:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 57:1 66:1 67:1 72:1 74:1 77:1 82:1 83:1
+-1 2:1 6:1 14:1 24:1 38:1 41:1 47:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 41:1 52:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 10:1 14:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 34:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 1:1 10:1 14:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 49:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 33:1 35:1 45:1 49:1 66:1 68:1 72:1 74:1 76:1 80:1 98:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 7:1 14:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 3:1 6:1 15:1 21:1 35:1 41:1 49:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 21:1 35:1 42:1 54:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 7:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 5:1 10:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 5:1 7:1 14:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 29:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 14:1 20:1 37:1 43:1 55:1 66:1 69:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 24:1 38:1 42:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 10:1 17:1 23:1 39:1 40:1 55:1 63:1 68:1 73:1 74:1 76:1 80:1 98:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 10:1 14:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 19:1 39:1 42:1 52:1 62:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 9:1 18:1 29:1 39:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 24:1 38:1 42:1 51:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 21:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1
+-1 4:1 7:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 26:1 35:1 42:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 8:1 15:1 24:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 87:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 9:1 14:1 25:1 38:1 40:1 55:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 11:1 18:1 22:1 36:1 41:1 49:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 24:1 38:1 41:1 48:1 64:1 71:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 16:1 33:1 35:1 42:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 5:1 6:1 16:1 26:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 40:1 50:1 65:1 68:1 72:1 74:1 76:1 80:1 92:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 24:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 99:1
+-1 2:1 17:1 22:1 36:1 42:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 52:1 65:1 68:1 72:1 74:1 76:1 78:1 93:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 17:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 25:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 16:1 31:1 35:1 40:1 56:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 33:1 35:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 53:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 18:1 22:1 36:1 41:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 15:1 33:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 41:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 18:1 19:1 39:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 11:1 17:1 20:1 37:1 40:1 55:1 63:1 71:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 7:1 16:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 43:1 48:1 66:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 40:1 54:1 61:1 67:1 72:1 75:1 76:1 80:1 119:1
++1 3:1 6:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 18:1 19:1 39:1 40:1 54:1 63:1 68:1 73:1 75:1 76:1 80:1
+-1 4:1 14:1 20:1 37:1 41:1 64:1 69:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 14:1 26:1 35:1 42:1 49:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 28:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 8:1 14:1 22:1 36:1 45:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 14:1 21:1 35:1 42:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 24:1 38:1 42:1 52:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 18:1 20:1 37:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 57:1 62:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 11:1 14:1 29:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 7:1 18:1 31:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 4:1 10:1 16:1 27:1 35:1 45:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 86:1
+-1 3:1 6:1 14:1 20:1 37:1 43:1 49:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 44:1 56:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 57:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 8:1 17:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 6:1 15:1 25:1 38:1 40:1 47:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 17:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 8:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 11:1 14:1 19:1 39:1 45:1 52:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 10:1 15:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 17:1 31:1 35:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 9:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 15:1 24:1 38:1 43:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 68:1 73:1 74:1 76:1 80:1 98:1
+-1 2:1 6:1 15:1 32:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
++1 5:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 5:1 8:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 11:1 17:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 9:1 18:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 18:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 3:1 7:1 15:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 7:1 18:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 16:1 31:1 35:1 40:1 49:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 14:1 27:1 35:1 41:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 16:1 30:1 35:1 44:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 53:1 63:1 69:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 43:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 18:1 19:1 39:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 41:1 48:1 64:1 71:1 73:1 75:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 26:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 5:1 8:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 17:1 21:1 35:1 44:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 10:1 17:1 20:1 37:1 41:1 49:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 15:1 22:1 36:1 41:1 64:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 21:1 35:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 10:1 17:1 19:1 39:1 42:1 52:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 48:1 65:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 31:1 35:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 42:1 51:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 43:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1
+-1 5:1 6:1 18:1 28:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 112:1
++1 2:1 6:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 19:1 39:1 44:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 27:1 35:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 31:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 14:1 22:1 36:1 42:1 62:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 26:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 24:1 38:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 8:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1
++1 3:1 8:1 14:1 25:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 17:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 28:1 35:1 42:1 52:1 64:1 68:1 72:1 74:1 76:1 78:1 117:1
++1 4:1 6:1 15:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 16:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 34:1 35:1 42:1 49:1 64:1 68:1 72:1 74:1 76:1 80:1 108:1
+-1 3:1 6:1 17:1 22:1 36:1 41:1 47:1 62:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 8:1 16:1 19:1 39:1 42:1 51:1 62:1 68:1 73:1 74:1 76:1 79:1 110:1
++1 3:1 11:1 18:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 88:1
+-1 2:1 7:1 14:1 29:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 85:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 41:1 49:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 5:1 8:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 4:1 6:1 18:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 25:1 38:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 18:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 29:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 1:1 6:1 14:1 28:1 35:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 11:1 17:1 19:1 39:1 41:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 45:1 51:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 17:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 43:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 41:1 50:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 29:1 39:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 10:1 16:1 19:1 39:1 42:1 59:1 64:1 67:1 72:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 21:1 35:1 42:1 57:1 65:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 100:1
+-1 4:1 10:1 17:1 22:1 36:1 43:1 49:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 85:1
++1 3:1 11:1 14:1 20:1 37:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 79:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 47:1 64:1 71:1 72:1 74:1 76:1 80:1 106:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 15:1 22:1 36:1 42:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 2:1 6:1 15:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 80:1 103:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 44:1 54:1 66:1 71:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 17:1 22:1 36:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 17:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 47:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 8:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 4:1 11:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 8:1 16:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 44:1 53:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 29:1 39:1 44:1 52:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 5:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 47:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 14:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 7:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 10:1 14:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 4:1 10:1 16:1 19:1 39:1 40:1 52:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 14:1 31:1 35:1 45:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 7:1 15:1 31:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 24:1 38:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 68:1 73:1 74:1 76:1 80:1 110:1
+-1 5:1 6:1 16:1 20:1 37:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 14:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 9:1 18:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 14:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 31:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 11:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 18:1 25:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 27:1 35:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 16:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 27:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 15:1 25:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 92:1
+-1 4:1 10:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 11:1 15:1 22:1 36:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 40:1 50:1 61:1 67:1 73:1 74:1 76:1 82:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 41:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 9:1 14:1 22:1 36:1 42:1 55:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 3:1 6:1 14:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 24:1 38:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 17:1 29:1 39:1 40:1 52:1 63:1 68:1 73:1 74:1 76:1 78:1 90:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 19:1 39:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 15:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 2:1 10:1 17:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 42:1 48:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 15:1 22:1 36:1 43:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 17:1 22:1 36:1 44:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 21:1 35:1 43:1 56:1 65:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 1:1 10:1 16:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 29:1 39:1 40:1 52:1 63:1 68:1 73:1 74:1 76:1 81:1 90:1
++1 5:1 9:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 104:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 27:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 29:1 39:1 42:1 51:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 43:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 11:1 16:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 9:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 23:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 18:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 59:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 18:1 19:1 39:1 44:1 52:1 64:1 71:1 72:1 74:1 77:1 78:1 83:1
++1 5:1 6:1 16:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 16:1 27:1 35:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 88:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 7:1 14:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 15:1 19:1 39:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 55:1 66:1 71:1 72:1 75:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 19:1 39:1 43:1 51:1 66:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 53:1 62:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 47:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 18:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 54:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 33:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 2:1 9:1 18:1 20:1 37:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 10:1 17:1 19:1 39:1 44:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 19:1 39:1 44:1 50:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 19:1 39:1 43:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 44:1 52:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 27:1 35:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 87:1
+-1 5:1 6:1 16:1 29:1 39:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 10:1 14:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 21:1 35:1 42:1 48:1 64:1 68:1 73:1 74:1 76:1 80:1 90:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 15:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 10:1 18:1 20:1 37:1 41:1 59:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 53:1 65:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 3:1 11:1 14:1 29:1 39:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 43:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 26:1 35:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 119:1
++1 4:1 8:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 24:1 38:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 78:1 83:1
++1 2:1 10:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 11:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 11:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 11:1 17:1 19:1 39:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 48:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 11:1 14:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 49:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 20:1 37:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 64:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 47:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 3:1 7:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 8:1 16:1 22:1 36:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 41:1 51:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 41:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 17:1 29:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 99:1
++1 5:1 9:1 15:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 10:1 17:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 25:1 38:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 57:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 41:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 40:1 53:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 29:1 39:1 42:1 51:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 1:1 6:1 18:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 26:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 2:1 7:1 14:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 18:1 21:1 35:1 42:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 9:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 90:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 48:1 65:1 68:1 72:1 74:1 76:1 80:1 102:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 7:1 14:1 20:1 37:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 21:1 35:1 40:1 53:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 17:1 21:1 35:1 42:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 8:1 15:1 22:1 36:1 42:1 50:1 65:1 67:1 73:1 74:1 76:1 80:1
++1 5:1 11:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 4:1 6:1 18:1 27:1 35:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 18:1 20:1 37:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 31:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 14:1 21:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 53:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 14:1 19:1 39:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 18:1 25:1 38:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 31:1 35:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 53:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 14:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 17:1 29:1 39:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 11:1 15:1 24:1 38:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 16:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 25:1 38:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 14:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 7:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 8:1 15:1 19:1 39:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 8:1 16:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1
++1 4:1 8:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 11:1 18:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 21:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 109:1
+-1 3:1 6:1 15:1 20:1 37:1 43:1 49:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 41:1 48:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 7:1 14:1 19:1 39:1 41:1 56:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 27:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 50:1 64:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 29:1 39:1 42:1 52:1 64:1 68:1 73:1 74:1 76:1 78:1 90:1
+-1 5:1 6:1 15:1 22:1 36:1 43:1 57:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 11:1 17:1 19:1 39:1 41:1 55:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 15:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 14:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 41:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 15:1 24:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 10:1 15:1 24:1 38:1 42:1 55:1 62:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 40:1 52:1 63:1 71:1 73:1 74:1 76:1 82:1
+-1 3:1 7:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 18:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 11:1 14:1 19:1 39:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 18:1 22:1 36:1 41:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 41:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 44:1 52:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 41:1 57:1 66:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 11:1 16:1 19:1 39:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 17:1 20:1 37:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 24:1 38:1 41:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 8:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 26:1 35:1 40:1 56:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 55:1 65:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 57:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 27:1 35:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 17:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 87:1
++1 4:1 9:1 18:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 25:1 38:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 19:1 39:1 41:1 52:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 10:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 8:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 41:1 54:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 18:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 10:1 14:1 32:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 21:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 17:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 26:1 35:1 40:1 49:1 63:1 71:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 17:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 4:1 8:1 14:1 29:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1
+-1 5:1 11:1 17:1 27:1 35:1 40:1 49:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 54:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 55:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 17:1 22:1 36:1 41:1 50:1 62:1 67:1 73:1 74:1 76:1 82:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 48:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 15:1 24:1 38:1 41:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
\ No newline at end of file
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_3.libsvm b/src/test/scripts/functions/io/libsvm/in/transfusion_3.libsvm
new file mode 100644
index 0000000..acff4f5
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_3.libsvm
@@ -0,0 +1,6 @@
+1 1:0.25 2:1.
+2 1:0.4 2:0.5
+3 1:0.5 2:0.15
+4 1:0.5 2:0.15
+5 1:0.5 2:0.15
+6 1:0.5 2:0.15
\ No newline at end of file
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_W1.libsvm b/src/test/scripts/functions/io/libsvm/in/transfusion_W1.libsvm
new file mode 100644
index 0000000..1bdb40b
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_W1.libsvm
@@ -0,0 +1,6 @@
+1 1:5 3:9
+2 1:5 2:9
+3 1:5 2:9
+4 1:5 2:9
+5 1:5 3:9
+6 1:5 2:9
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_W1.libsvm.mtd b/src/test/scripts/functions/io/libsvm/in/transfusion_W1.libsvm.mtd
new file mode 100644
index 0000000..689c84f
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_W1.libsvm.mtd
@@ -0,0 +1,11 @@
+{
+ "data_type": "matrix",
+ "value_type": "double",
+ "rows": 6,
+ "cols": 3,
+ "sep": " ",
+ "indSep": ":",
+ "format": "libsvm",
+ "author": "Saeed Fathollahzadeh",
+ "created": "2021-04-14 17:27:25 CEST"
+}
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_W2.libsvm b/src/test/scripts/functions/io/libsvm/in/transfusion_W2.libsvm
new file mode 100644
index 0000000..150ee74
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_W2.libsvm
@@ -0,0 +1,1605 @@
+-1 3:1 11:1 14:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 27:1 35:1 40:1 57:1 63:1 69:1 73:1 74:1 76:1 81:1 103:1
+-1 4:1 6:1 15:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 47:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 41:1 47:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 48:1 64:1 71:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 11:1 15:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 16:1 30:1 35:1 41:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 29:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 11:1 18:1 20:1 37:1 42:1 59:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 18:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 18:1 20:1 37:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 107:1
+-1 1:1 6:1 14:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 44:1 51:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 10:1 18:1 22:1 36:1 42:1 56:1 64:1 67:1 73:1 74:1 76:1 80:1 100:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 27:1 35:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 10:1 18:1 22:1 36:1 43:1 51:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 7:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 15:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 15:1 21:1 35:1 45:1 62:1 68:1 72:1 74:1 77:1 80:1 93:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 55:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 5:1 6:1 16:1 20:1 37:1 41:1 55:1 64:1 68:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 25:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 42:1 50:1 64:1 70:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
++1 4:1 6:1 14:1 29:1 39:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 26:1 35:1 41:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 16:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 51:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 22:1 36:1 42:1 56:1 66:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 6:1 16:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 3:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 87:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 42:1 50:1 64:1 71:1 72:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 25:1 38:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 16:1 22:1 36:1 41:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 17:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 7:1 17:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 44:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 16:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 43:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 24:1 38:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 1:1 6:1 14:1 25:1 38:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 55:1 62:1 68:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 11:1 14:1 22:1 36:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 71:1 72:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 15:1 26:1 35:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 87:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 75:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 49:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 48:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 16:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 49:1 66:1 67:1 73:1 74:1 76:1 80:1 119:1
++1 4:1 6:1 14:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 10:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 48:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 42:1 56:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 20:1 37:1 42:1 55:1 65:1 71:1 72:1 74:1 76:1 80:1 101:1
+-1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 17:1 25:1 38:1 41:1 50:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 9:1 16:1 24:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 44:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 16:1 19:1 39:1 43:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 81:1 83:1
++1 3:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 10:1 17:1 19:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 15:1 29:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 10:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 43:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 14:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 68:1 73:1 74:1 76:1 81:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 33:1 35:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 15:1 29:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 9:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 9:1 18:1 21:1 35:1 42:1 47:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 11:1 18:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 17:1 20:1 37:1 43:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 95:1
+-1 3:1 6:1 18:1 23:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 77:1 80:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 9:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 59:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 9:1 14:1 22:1 36:1 46:1 50:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 43:1 55:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 10:1 15:1 29:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 34:1 35:1 42:1 49:1 65:1 67:1 72:1 74:1 76:1 79:1 103:1
+-1 1:1 18:1 22:1 36:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 10:1 17:1 21:1 35:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 16:1 22:1 36:1 42:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 11:1 15:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 17:1 25:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 88:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 40:1 50:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 14:1 24:1 38:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 45:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 100:1
++1 2:1 10:1 18:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 4:1 6:1 14:1 20:1 37:1 42:1 56:1 65:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 21:1 35:1 41:1 48:1 65:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 43:1 49:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 16:1 22:1 36:1 40:1 63:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 25:1 38:1 40:1 54:1 63:1 69:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 11:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 10:1 18:1 24:1 38:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 8:1 18:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 47:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 16:1 29:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 53:1 62:1 71:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 16:1 19:1 39:1 41:1 51:1 64:1 71:1 73:1 75:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 87:1
+-1 4:1 6:1 17:1 33:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 79:1 103:1
+-1 2:1 6:1 18:1 20:1 37:1 41:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 10:1 15:1 22:1 36:1 42:1 59:1 65:1 71:1 73:1 74:1 77:1 80:1 83:1
++1 3:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 11:1 15:1 19:1 39:1 42:1 52:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 4:1 6:1 14:1 25:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 6:1 16:1 20:1 37:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 9:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 16:1 20:1 37:1 40:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 16:1 20:1 37:1 42:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 49:1 63:1 71:1 73:1 74:1 76:1 82:1 85:1
+-1 5:1 6:1 14:1 29:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 14:1 22:1 36:1 43:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 18:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 53:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 11:1 15:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 8:1 16:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 19:1 39:1 43:1 51:1 64:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 68:1 72:1 74:1 76:1 80:1 102:1
+-1 5:1 6:1 14:1 22:1 36:1 44:1 49:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 42:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 16:1 28:1 35:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1
+-1 5:1 6:1 15:1 20:1 37:1 44:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 99:1
+-1 2:1 7:1 17:1 22:1 36:1 41:1 56:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 40:1 50:1 63:1 70:1 73:1 74:1 76:1 82:1
++1 5:1 6:1 15:1 24:1 38:1 40:1 51:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 9:1 14:1 25:1 38:1 42:1 52:1 65:1 68:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 10:1 15:1 19:1 39:1 42:1 52:1 64:1 71:1 73:1 75:1 76:1 81:1
++1 5:1 7:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 27:1 35:1 42:1 56:1 66:1 70:1 73:1 74:1 76:1 79:1 103:1
+-1 4:1 7:1 14:1 20:1 37:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 48:1 65:1 68:1 72:1 74:1 76:1 80:1 84:1
++1 4:1 8:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 41:1 49:1 66:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 8:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 10:1 17:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 31:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 79:1 103:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 10:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 7:1 18:1 30:1 35:1 45:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 4:1 6:1 18:1 30:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 22:1 36:1 41:1 57:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 15:1 25:1 38:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 10:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 17:1 29:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 26:1 35:1 43:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 119:1
++1 1:1 6:1 16:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 18:1 29:1 39:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 119:1
++1 3:1 11:1 14:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 79:1 83:1
+-1 5:1 6:1 15:1 33:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1
++1 4:1 6:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 8:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 43:1 57:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 7:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 88:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 7:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 41:1 48:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 58:1 66:1 71:1 72:1 74:1 76:1 78:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 54:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 50:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 10:1 16:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 17:1 19:1 39:1 43:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 41:1 54:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 8:1 15:1 20:1 37:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 11:1 18:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 17:1 20:1 37:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 10:1 15:1 20:1 37:1 43:1 59:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 31:1 35:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 32:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 7:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 18:1 22:1 36:1 44:1 51:1 66:1 68:1 72:1 74:1 76:1 82:1 93:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 49:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 15:1 27:1 35:1 44:1 55:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 11:1 17:1 23:1 39:1 42:1 52:1 64:1 68:1 73:1 75:1 76:1 82:1 90:1
+-1 3:1 7:1 15:1 20:1 37:1 40:1 48:1 63:1 69:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 14:1 27:1 35:1 45:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 99:1
+-1 3:1 17:1 25:1 38:1 42:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 33:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 78:1 103:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 47:1 64:1 71:1 73:1 74:1 76:1 80:1
+-1 2:1 6:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 17:1 20:1 37:1 41:1 51:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 18:1 21:1 35:1 41:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 27:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 43:1 51:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 9:1 14:1 20:1 37:1 41:1 47:1 66:1 69:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 19:1 39:1 42:1 55:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 49:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 82:1 88:1
+-1 1:1 16:1 25:1 38:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 2:1 7:1 17:1 19:1 39:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 11:1 18:1 19:1 39:1 41:1 59:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 47:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 7:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 18:1 29:1 39:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 107:1
+-1 3:1 6:1 15:1 22:1 36:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 21:1 35:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 119:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 41:1 55:1 64:1 68:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 16:1 33:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 1:1 6:1 18:1 27:1 35:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 9:1 15:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 17:1 19:1 39:1 43:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 24:1 38:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 14:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 25:1 38:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 17:1 23:1 39:1 41:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 18:1 31:1 35:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 23:1 39:1 43:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 7:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 26:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 9:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 22:1 36:1 40:1 50:1 63:1 68:1 73:1 74:1 76:1 82:1 93:1
+-1 5:1 6:1 17:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 2:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 50:1 62:1 68:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 18:1 19:1 39:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 8:1 15:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 58:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 25:1 38:1 42:1 51:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 27:1 35:1 44:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 16:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 14:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 10:1 18:1 28:1 35:1 42:1 49:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 7:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 42:1 52:1 66:1 67:1 72:1 74:1 77:1 80:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 46:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 25:1 38:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 16:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 41:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 100:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 5:1 7:1 16:1 22:1 36:1 42:1 56:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 10:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 9:1 16:1 23:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 2:1 7:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 40:1 55:1 61:1 69:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 14:1 23:1 39:1 40:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 7:1 17:1 20:1 37:1 41:1 56:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 14:1 29:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 32:1 39:1 41:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 77:1 78:1 83:1
++1 3:1 6:1 14:1 19:1 39:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 77:1 78:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 110:1
+-1 3:1 6:1 15:1 20:1 37:1 43:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 25:1 38:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 14:1 25:1 38:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 68:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 33:1 35:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 78:1 103:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 70:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 26:1 35:1 40:1 54:1 65:1 68:1 72:1 74:1 76:1 81:1 94:1
+-1 1:1 6:1 15:1 25:1 38:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 10:1 18:1 31:1 35:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 11:1 16:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 17:1 29:1 39:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 57:1 64:1 69:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 2:1 10:1 16:1 24:1 38:1 40:1 59:1 63:1 67:1 73:1 74:1 77:1 80:1 85:1
+-1 4:1 6:1 15:1 22:1 36:1 44:1 50:1 66:1 69:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 10:1 14:1 22:1 36:1 40:1 51:1 63:1 69:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 7:1 16:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 18:1 20:1 37:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 41:1 49:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 16:1 21:1 35:1 42:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 31:1 35:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 11:1 17:1 29:1 39:1 40:1 52:1 61:1 67:1 72:1 75:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 18:1 22:1 36:1 44:1 49:1 65:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 55:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 15:1 23:1 39:1 45:1 66:1 68:1 72:1 74:1 76:1 80:1 102:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 14:1 19:1 39:1 41:1 52:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 43:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 16:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 31:1 35:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 11:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 112:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 53:1 61:1 67:1 72:1 74:1 76:1 80:1 103:1
++1 5:1 7:1 14:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 85:1
+-1 2:1 10:1 18:1 19:1 39:1 42:1 59:1 62:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 54:1 65:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 2:1 6:1 14:1 20:1 37:1 40:1 49:1 61:1 69:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 66:1 71:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 31:1 35:1 40:1 48:1 61:1 70:1 72:1 74:1 76:1 82:1 114:1
+-1 4:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 54:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1
+-1 3:1 8:1 15:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 16:1 24:1 38:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 21:1 35:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 56:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 27:1 35:1 42:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 9:1 14:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 82:1 105:1
+-1 2:1 7:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 15:1 34:1 35:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 107:1
+-1 2:1 6:1 15:1 26:1 35:1 40:1 56:1 63:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 8:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 10:1 17:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 68:1 73:1 74:1 76:1 80:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 86:1
+-1 5:1 7:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 48:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 8:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 9:1 17:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 14:1 25:1 38:1 41:1 47:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 17:1 19:1 39:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 11:1 15:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 18:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 21:1 35:1 42:1 49:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 14:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 10:1 17:1 22:1 36:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 79:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 41:1 54:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 48:1 64:1 71:1 72:1 74:1 76:1 81:1 83:1
++1 5:1 9:1 16:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 43:1 57:1 65:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 15:1 22:1 36:1 43:1 48:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 31:1 35:1 42:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 44:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 29:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 10:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 2:1 14:1 20:1 37:1 43:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 21:1 35:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 27:1 35:1 41:1 53:1 65:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 7:1 14:1 22:1 36:1 41:1 56:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 58:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 3:1 11:1 14:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 31:1 35:1 42:1 50:1 66:1 67:1 72:1 74:1 76:1 79:1
+-1 5:1 10:1 16:1 33:1 35:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 45:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 14:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 33:1 35:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 112:1
+-1 3:1 6:1 17:1 19:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 10:1 14:1 20:1 37:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 27:1 35:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 29:1 39:1 42:1 51:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 15:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 85:1
+-1 5:1 6:1 17:1 33:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 4:1 10:1 15:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 52:1 62:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 49:1 66:1 71:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 50:1 65:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 24:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 10:1 16:1 26:1 35:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 118:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 43:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 7:1 15:1 22:1 36:1 41:1 51:1 64:1 67:1 72:1 75:1 76:1 79:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 7:1 18:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 18:1 20:1 37:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 52:1 64:1 71:1 72:1 75:1 76:1 82:1 83:1
++1 4:1 6:1 16:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 20:1 37:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 18:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 8:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 17:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 43:1 49:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 15:1 22:1 36:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 16:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 42:1 56:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 15:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 20:1 37:1 40:1 56:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 16:1 22:1 36:1 42:1 52:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 44:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 11:1 16:1 29:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 33:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 2:1 18:1 22:1 36:1 42:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 40:1 53:1 63:1 71:1 73:1 74:1 76:1 80:1
++1 5:1 6:1 14:1 19:1 39:1 42:1 49:1 64:1 68:1 72:1 74:1 76:1 78:1 91:1
+-1 5:1 6:1 16:1 34:1 35:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 7:1 18:1 22:1 36:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 17:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 62:1 69:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 3:1 7:1 16:1 19:1 39:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 28:1 35:1 42:1 57:1 64:1 68:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 7:1 16:1 20:1 37:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 24:1 38:1 42:1 52:1 64:1 68:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 29:1 39:1 43:1 51:1 64:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 11:1 16:1 20:1 37:1 42:1 49:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 15:1 23:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 7:1 14:1 24:1 38:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 42:1 54:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 42:1 54:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 9:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 11:1 18:1 19:1 39:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 14:1 30:1 35:1 42:1 49:1 65:1 67:1 72:1 74:1 76:1 78:1 119:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 9:1 15:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 14:1 19:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 58:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 50:1 64:1 71:1 73:1 74:1 76:1 80:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 48:1 66:1 67:1 72:1 74:1 76:1 80:1 86:1
++1 4:1 10:1 17:1 19:1 39:1 41:1 59:1 64:1 71:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 42:1 50:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 16:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 29:1 39:1 42:1 51:1 64:1 70:1 73:1 74:1 76:1 78:1 95:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 53:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 28:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 23:1 39:1 42:1 52:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 31:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 8:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 2:1 6:1 14:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 44:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 77:1 78:1 83:1
++1 4:1 9:1 17:1 20:1 37:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 17:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 18:1 27:1 35:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 26:1 35:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 26:1 35:1 43:1 53:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 3:1 6:1 16:1 20:1 37:1 40:1 54:1 63:1 68:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 16:1 30:1 35:1 40:1 49:1 65:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 50:1 62:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 16:1 24:1 38:1 42:1 65:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 8:1 16:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 14:1 23:1 39:1 45:1 52:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 18:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 10:1 18:1 19:1 39:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 9:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 18:1 20:1 37:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 18:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 17:1 25:1 38:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 13:1 17:1 31:1 35:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 77:1 79:1 83:1
+-1 3:1 10:1 15:1 25:1 38:1 42:1 51:1 66:1 69:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 25:1 38:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 8:1 16:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 44:1 58:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 31:1 35:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 25:1 38:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 52:1 64:1 67:1 72:1 74:1 77:1 81:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 51:1 62:1 68:1 72:1 74:1 76:1 82:1 110:1
++1 4:1 7:1 18:1 19:1 39:1 41:1 49:1 66:1 68:1 72:1 74:1 76:1 82:1 93:1
+-1 1:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 10:1 15:1 19:1 39:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 6:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 8:1 18:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 16:1 31:1 35:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 95:1
++1 3:1 7:1 17:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 40:1 58:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 17:1 29:1 39:1 40:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 17:1 25:1 38:1 42:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 18:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 31:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 18:1 26:1 35:1 43:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 9:1 17:1 25:1 38:1 41:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 18:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 77:1 80:1 83:1
++1 3:1 6:1 16:1 24:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 9:1 14:1 20:1 37:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 25:1 38:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 2:1 6:1 17:1 22:1 36:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 49:1 62:1 69:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 8:1 15:1 20:1 37:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 30:1 35:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 16:1 23:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 15:1 25:1 38:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 17:1 20:1 37:1 41:1 57:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 8:1 17:1 22:1 36:1 40:1 50:1 63:1 71:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 5:1 15:1 22:1 36:1 44:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 40:1 55:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 8:1 14:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 40:1 55:1 62:1 69:1 72:1 75:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 27:1 35:1 45:1 54:1 64:1 67:1 73:1 74:1 76:1 80:1 114:1
++1 1:1 6:1 15:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 100:1
++1 4:1 10:1 15:1 29:1 39:1 40:1 52:1 63:1 71:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 15:1 21:1 35:1 42:1 53:1 65:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 24:1 38:1 41:1 55:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 14:1 31:1 35:1 44:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 11:1 17:1 32:1 39:1 43:1 52:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 1:1 7:1 18:1 26:1 35:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 79:1 103:1
++1 4:1 9:1 18:1 20:1 37:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 19:1 39:1 42:1 55:1 62:1 68:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 24:1 38:1 40:1 50:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 4:1 17:1 25:1 38:1 41:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 41:1 50:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1
+-1 5:1 6:1 15:1 20:1 37:1 41:1 54:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 29:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 109:1
++1 3:1 6:1 14:1 19:1 39:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 41:1 55:1 65:1 69:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 27:1 35:1 44:1 49:1 66:1 69:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 15:1 19:1 39:1 42:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 16:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 9:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 10:1 15:1 20:1 37:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 100:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 51:1 64:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 18:1 19:1 39:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 19:1 39:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 17:1 29:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 16:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 5:1 7:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 24:1 38:1 42:1 49:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 26:1 35:1 42:1 48:1 66:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 16:1 29:1 39:1 42:1 55:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 42:1 51:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 21:1 35:1 42:1 49:1 62:1 71:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 10:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 11:1 15:1 22:1 36:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 15:1 26:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 26:1 35:1 40:1 54:1 63:1 71:1 73:1 75:1 76:1 80:1 83:1
++1 2:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 25:1 38:1 40:1 47:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 3:1 11:1 17:1 20:1 37:1 43:1 51:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 79:1 83:1
++1 5:1 6:1 14:1 22:1 36:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 31:1 35:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 17:1 20:1 37:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 78:1 87:1
+-1 5:1 6:1 16:1 31:1 35:1 44:1 56:1 66:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 87:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 42:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 100:1
+-1 5:1 6:1 18:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 14:1 27:1 35:1 40:1 50:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 21:1 35:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 9:1 15:1 19:1 39:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 15:1 20:1 37:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 16:1 19:1 39:1 42:1 52:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 41:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 29:1 39:1 41:1 59:1 64:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 22:1 36:1 43:1 49:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 33:1 35:1 42:1 53:1 65:1 67:1 73:1 74:1 76:1 78:1 97:1
+-1 4:1 6:1 16:1 22:1 36:1 43:1 52:1 66:1 67:1 72:1 74:1 77:1 80:1 86:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 27:1 35:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 78:1 95:1
+-1 3:1 6:1 17:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 44:1 51:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 3:1 6:1 17:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 26:1 35:1 40:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 10:1 18:1 20:1 37:1 42:1 51:1 66:1 71:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 43:1 50:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 41:1 53:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 42:1 48:1 65:1 68:1 73:1 74:1 76:1 80:1 84:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 54:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 29:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 82:1 99:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 53:1 62:1 68:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 11:1 15:1 22:1 36:1 42:1 57:1 64:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 55:1 64:1 68:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 18:1 26:1 35:1 41:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 7:1 15:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 18:1 25:1 38:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 24:1 38:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 31:1 35:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 41:1 55:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 18:1 33:1 35:1 41:1 53:1 66:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 3:1 6:1 14:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 47:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 18:1 22:1 36:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 9:1 18:1 20:1 37:1 40:1 47:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
++1 5:1 7:1 16:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 51:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 8:1 15:1 29:1 39:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 54:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 9:1 14:1 19:1 39:1 45:1 51:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 21:1 35:1 45:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 18:1 29:1 39:1 41:1 51:1 64:1 71:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 16:1 25:1 38:1 42:1 48:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 24:1 38:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 78:1 113:1
+-1 4:1 6:1 17:1 33:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
++1 3:1 6:1 16:1 19:1 39:1 40:1 50:1 61:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 4:1 11:1 15:1 22:1 36:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 25:1 38:1 43:1 52:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 45:1 57:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 6:1 17:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 10:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 50:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 42:1 55:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 44:1 49:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 8:1 16:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 11:1 18:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 16:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 16:1 19:1 39:1 42:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 10:1 15:1 22:1 36:1 41:1 57:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 16:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 7:1 18:1 20:1 37:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 16:1 22:1 36:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 10:1 14:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 10:1 15:1 22:1 36:1 40:1 55:1 61:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 11:1 17:1 19:1 39:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 25:1 38:1 42:1 59:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 16:1 33:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 87:1
+-1 1:1 6:1 18:1 19:1 39:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 1:1 6:1 14:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 77:1 80:1 83:1
++1 2:1 7:1 14:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
++1 5:1 6:1 17:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 26:1 35:1 40:1 58:1 61:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 21:1 35:1 40:1 53:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 3:1 7:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 41:1 55:1 66:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 16:1 31:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 7:1 16:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 7:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 42:1 57:1 64:1 71:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 10:1 16:1 20:1 37:1 45:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 88:1
+-1 1:1 16:1 20:1 37:1 42:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 14:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 10:1 18:1 25:1 38:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 31:1 35:1 40:1 50:1 61:1 68:1 72:1 74:1 76:1 78:1 91:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 54:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 49:1 63:1 68:1 73:1 74:1 76:1 80:1 94:1
+-1 4:1 6:1 18:1 22:1 36:1 41:1 50:1 66:1 71:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 11:1 16:1 32:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 9:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 33:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 4:1 10:1 14:1 27:1 35:1 41:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 24:1 38:1 41:1 51:1 64:1 71:1 72:1 74:1 76:1 80:1 101:1
+-1 5:1 7:1 16:1 22:1 36:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 48:1 63:1 68:1 73:1 74:1 76:1 80:1 84:1
+-1 3:1 9:1 14:1 22:1 36:1 42:1 55:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 21:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 43:1 50:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 18:1 21:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 103:1
++1 4:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 15:1 20:1 37:1 43:1 50:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 15:1 19:1 39:1 42:1 52:1 66:1 70:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 49:1 64:1 68:1 72:1 74:1 76:1 79:1 117:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 53:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 14:1 20:1 37:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 20:1 37:1 45:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 11:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 18:1 19:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 45:1 55:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 10:1 15:1 24:1 38:1 41:1 49:1 64:1 67:1 72:1 75:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 55:1 65:1 67:1 73:1 74:1 76:1 78:1 115:1
+-1 5:1 6:1 18:1 31:1 35:1 44:1 54:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 7:1 17:1 19:1 39:1 41:1 50:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 119:1
+-1 4:1 6:1 17:1 20:1 37:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 24:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 18:1 24:1 38:1 40:1 51:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 24:1 38:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 5:1 10:1 14:1 19:1 39:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 14:1 19:1 39:1 40:1 56:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 42:1 48:1 62:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 25:1 38:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 10:1 17:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 9:1 18:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 17:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 14:1 31:1 35:1 40:1 63:1 69:1 73:1 75:1 76:1 78:1 83:1
+-1 5:1 15:1 20:1 37:1 41:1 64:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 57:1 66:1 67:1 72:1 74:1 77:1 82:1 83:1
+-1 2:1 6:1 14:1 24:1 38:1 41:1 47:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 16:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 41:1 52:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 10:1 14:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 34:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 1:1 10:1 14:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 49:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 33:1 35:1 45:1 49:1 66:1 68:1 72:1 74:1 76:1 80:1 98:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 7:1 14:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 3:1 6:1 15:1 21:1 35:1 41:1 49:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 21:1 35:1 42:1 54:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 7:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 5:1 10:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 54:1 64:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 5:1 7:1 14:1 20:1 37:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 29:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 14:1 20:1 37:1 43:1 55:1 66:1 69:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 24:1 38:1 42:1 54:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 10:1 17:1 23:1 39:1 40:1 55:1 63:1 68:1 73:1 74:1 76:1 80:1 98:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 10:1 14:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 19:1 39:1 42:1 52:1 62:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 9:1 18:1 29:1 39:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 11:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 24:1 38:1 42:1 51:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 21:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1
+-1 4:1 7:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 26:1 35:1 42:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 8:1 15:1 24:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 87:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 9:1 14:1 25:1 38:1 40:1 55:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 11:1 18:1 22:1 36:1 41:1 49:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 24:1 38:1 41:1 48:1 64:1 71:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 16:1 33:1 35:1 42:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 5:1 6:1 16:1 26:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 40:1 50:1 65:1 68:1 72:1 74:1 76:1 80:1 92:1
+-1 1:1 6:1 15:1 28:1 35:1 42:1 57:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 24:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 99:1
+-1 2:1 17:1 22:1 36:1 42:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 52:1 65:1 68:1 72:1 74:1 76:1 78:1 93:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 17:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 25:1 38:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 16:1 31:1 35:1 40:1 56:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 14:1 33:1 35:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 19:1 39:1 42:1 53:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 18:1 22:1 36:1 41:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 15:1 33:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 41:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 18:1 19:1 39:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 16:1 20:1 37:1 42:1 54:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 11:1 17:1 20:1 37:1 40:1 55:1 63:1 71:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 7:1 16:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 43:1 48:1 66:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 18:1 22:1 36:1 40:1 54:1 61:1 67:1 72:1 75:1 76:1 80:1 119:1
++1 3:1 6:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 18:1 19:1 39:1 40:1 54:1 63:1 68:1 73:1 75:1 76:1 80:1
+-1 4:1 14:1 20:1 37:1 41:1 64:1 69:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 14:1 26:1 35:1 42:1 49:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 28:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 8:1 14:1 22:1 36:1 45:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 14:1 21:1 35:1 42:1 62:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 24:1 38:1 42:1 52:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 18:1 20:1 37:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 42:1 57:1 62:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 5:1 11:1 14:1 29:1 39:1 41:1 52:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 7:1 18:1 31:1 35:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 4:1 10:1 16:1 27:1 35:1 45:1 49:1 64:1 67:1 72:1 74:1 76:1 79:1 86:1
+-1 3:1 6:1 14:1 20:1 37:1 43:1 49:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 44:1 56:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 25:1 38:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 57:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 8:1 17:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 5:1 6:1 15:1 25:1 38:1 40:1 47:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 10:1 16:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 2:1 6:1 17:1 19:1 39:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 8:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 14:1 19:1 39:1 42:1 51:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 11:1 14:1 19:1 39:1 45:1 52:1 64:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 4:1 10:1 15:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 17:1 31:1 35:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 9:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 15:1 24:1 38:1 43:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 19:1 39:1 40:1 51:1 63:1 68:1 73:1 74:1 76:1 80:1 98:1
+-1 2:1 6:1 15:1 32:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 20:1 37:1 40:1 51:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
++1 5:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 5:1 8:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 11:1 17:1 19:1 39:1 42:1 52:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 9:1 18:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 14:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 18:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 3:1 7:1 15:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 7:1 18:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 16:1 31:1 35:1 40:1 49:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 14:1 27:1 35:1 41:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 16:1 30:1 35:1 44:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 2:1 6:1 14:1 22:1 36:1 40:1 53:1 63:1 69:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 43:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 18:1 19:1 39:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 41:1 48:1 64:1 71:1 73:1 75:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 26:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 16:1 24:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 5:1 8:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 5:1 17:1 21:1 35:1 44:1 65:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 10:1 17:1 20:1 37:1 41:1 49:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 15:1 22:1 36:1 41:1 64:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 21:1 35:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 10:1 17:1 19:1 39:1 42:1 52:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 48:1 65:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 31:1 35:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 42:1 51:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 43:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1
+-1 5:1 6:1 18:1 28:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 112:1
++1 2:1 6:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 19:1 39:1 44:1 51:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 20:1 37:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 14:1 27:1 35:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 20:1 37:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 31:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 14:1 22:1 36:1 42:1 62:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 26:1 35:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 24:1 38:1 40:1 59:1 63:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 8:1 17:1 27:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1
++1 3:1 8:1 14:1 25:1 38:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 17:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 28:1 35:1 42:1 52:1 64:1 68:1 72:1 74:1 76:1 78:1 117:1
++1 4:1 6:1 15:1 32:1 39:1 42:1 52:1 64:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 42:1 49:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 16:1 28:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 34:1 35:1 42:1 49:1 64:1 68:1 72:1 74:1 76:1 80:1 108:1
+-1 3:1 6:1 17:1 22:1 36:1 41:1 47:1 62:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 8:1 16:1 19:1 39:1 42:1 51:1 62:1 68:1 73:1 74:1 76:1 79:1 110:1
++1 3:1 11:1 18:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 88:1
+-1 2:1 7:1 14:1 29:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 81:1 85:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 41:1 49:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
++1 5:1 8:1 15:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 3:1 6:1 17:1 24:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 4:1 6:1 18:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 14:1 25:1 38:1 41:1 50:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 10:1 18:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 29:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 77:1 78:1 83:1
+-1 1:1 6:1 14:1 28:1 35:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 11:1 17:1 19:1 39:1 41:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 45:1 51:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 17:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 43:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 41:1 50:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 29:1 39:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 10:1 16:1 19:1 39:1 42:1 59:1 64:1 67:1 72:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 21:1 35:1 42:1 57:1 65:1 67:1 73:1 74:1 76:1 80:1 114:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 100:1
+-1 4:1 10:1 17:1 22:1 36:1 43:1 49:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 85:1
++1 3:1 11:1 14:1 20:1 37:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 79:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 47:1 64:1 71:1 72:1 74:1 76:1 80:1 106:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 15:1 22:1 36:1 42:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 2:1 6:1 15:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 75:1 76:1 80:1 103:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 14:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 44:1 54:1 66:1 71:1 72:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 17:1 22:1 36:1 40:1 57:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 17:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 47:1 66:1 71:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 8:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 4:1 11:1 14:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 18:1 22:1 36:1 40:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 17:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 5:1 8:1 16:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 44:1 53:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 29:1 39:1 44:1 52:1 64:1 67:1 72:1 75:1 76:1 82:1 83:1
+-1 5:1 16:1 22:1 36:1 40:1 63:1 67:1 73:1 75:1 76:1 78:1 83:1
+-1 5:1 6:1 16:1 21:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 49:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 47:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 14:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 7:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 10:1 14:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 4:1 10:1 16:1 19:1 39:1 40:1 52:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 62:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 14:1 31:1 35:1 45:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 7:1 15:1 31:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 18:1 20:1 37:1 42:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 22:1 36:1 40:1 55:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 24:1 38:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 68:1 73:1 74:1 76:1 80:1 110:1
+-1 5:1 6:1 16:1 20:1 37:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 14:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 2:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 9:1 18:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 14:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 22:1 36:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 15:1 31:1 35:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 11:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 18:1 25:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 27:1 35:1 41:1 54:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 16:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 14:1 27:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 15:1 25:1 38:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 92:1
+-1 4:1 10:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 11:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 11:1 15:1 22:1 36:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 19:1 39:1 40:1 50:1 61:1 67:1 73:1 74:1 76:1 82:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 57:1 62:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 41:1 55:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 9:1 14:1 22:1 36:1 42:1 55:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 3:1 6:1 14:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 24:1 38:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 17:1 29:1 39:1 40:1 52:1 63:1 68:1 73:1 74:1 76:1 78:1 90:1
+-1 3:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 19:1 39:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 15:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
++1 2:1 10:1 17:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 18:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 42:1 48:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 20:1 37:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 15:1 22:1 36:1 43:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 17:1 22:1 36:1 44:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 21:1 35:1 43:1 56:1 65:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 1:1 10:1 16:1 20:1 37:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 14:1 29:1 39:1 40:1 52:1 63:1 68:1 73:1 74:1 76:1 81:1 90:1
++1 5:1 9:1 14:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 104:1
+-1 3:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 42:1 57:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 27:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 29:1 39:1 42:1 51:1 64:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 17:1 22:1 36:1 43:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 11:1 16:1 20:1 37:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 9:1 14:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 15:1 23:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 18:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 59:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 10:1 18:1 19:1 39:1 44:1 52:1 64:1 71:1 72:1 74:1 77:1 78:1 83:1
++1 5:1 6:1 16:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 16:1 27:1 35:1 44:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 88:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 50:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 7:1 14:1 22:1 36:1 42:1 48:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 15:1 19:1 39:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 1:1 6:1 18:1 20:1 37:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 42:1 55:1 66:1 71:1 72:1 75:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 16:1 19:1 39:1 43:1 51:1 66:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 53:1 62:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 40:1 47:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 7:1 18:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 54:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 33:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 2:1 9:1 18:1 20:1 37:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 5:1 10:1 17:1 19:1 39:1 44:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 19:1 39:1 44:1 50:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 19:1 39:1 43:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 20:1 37:1 44:1 52:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 17:1 20:1 37:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 27:1 35:1 40:1 48:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 6:1 14:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 79:1 87:1
+-1 5:1 6:1 16:1 29:1 39:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 5:1 10:1 14:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 21:1 35:1 42:1 48:1 64:1 68:1 73:1 74:1 76:1 80:1 90:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 7:1 15:1 19:1 39:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 10:1 18:1 20:1 37:1 41:1 59:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 75:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 31:1 35:1 42:1 53:1 65:1 67:1 73:1 74:1 77:1 81:1 83:1
+-1 3:1 11:1 14:1 29:1 39:1 41:1 55:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 20:1 37:1 43:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 17:1 26:1 35:1 42:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 119:1
++1 4:1 8:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 24:1 38:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 78:1 83:1
++1 2:1 10:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 11:1 17:1 32:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 18:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 20:1 37:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 11:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 48:1 64:1 67:1 73:1 74:1 77:1 79:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 50:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 11:1 17:1 19:1 39:1 42:1 47:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 10:1 16:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 15:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 42:1 48:1 66:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 8:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 2:1 6:1 16:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 6:1 17:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 53:1 64:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 3:1 11:1 14:1 20:1 37:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 22:1 36:1 42:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 49:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 15:1 20:1 37:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 6:1 14:1 22:1 36:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 7:1 17:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 16:1 21:1 35:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 4:1 10:1 15:1 29:1 39:1 41:1 52:1 64:1 69:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 47:1 66:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 8:1 14:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 17:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 80:1 86:1
+-1 3:1 7:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 2:1 8:1 16:1 22:1 36:1 41:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 18:1 19:1 39:1 41:1 51:1 64:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 41:1 57:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 17:1 29:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 99:1
++1 5:1 9:1 15:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 5:1 10:1 17:1 22:1 36:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 25:1 38:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 57:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 17:1 22:1 36:1 41:1 52:1 62:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 17:1 20:1 37:1 40:1 53:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 29:1 39:1 42:1 51:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 2:1 6:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 18:1 20:1 37:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 49:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 1:1 6:1 18:1 20:1 37:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 18:1 26:1 35:1 40:1 56:1 63:1 67:1 73:1 74:1 76:1 82:1 103:1
+-1 2:1 7:1 14:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 48:1 62:1 71:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 18:1 21:1 35:1 42:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
++1 4:1 9:1 17:1 23:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 90:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 48:1 65:1 68:1 72:1 74:1 76:1 80:1 102:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 49:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 79:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 79:1 83:1
+-1 1:1 7:1 14:1 20:1 37:1 42:1 47:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 17:1 21:1 35:1 40:1 53:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 1:1 17:1 21:1 35:1 42:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 8:1 15:1 22:1 36:1 42:1 50:1 65:1 67:1 73:1 74:1 76:1 80:1
++1 5:1 11:1 15:1 22:1 36:1 40:1 51:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 4:1 6:1 18:1 27:1 35:1 41:1 50:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 18:1 20:1 37:1 40:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 6:1 18:1 31:1 35:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 14:1 21:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 15:1 22:1 36:1 42:1 53:1 62:1 71:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 7:1 14:1 19:1 39:1 42:1 49:1 64:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 18:1 25:1 38:1 42:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 5:1 6:1 15:1 31:1 35:1 40:1 48:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 42:1 53:1 64:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 16:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 10:1 14:1 19:1 39:1 40:1 52:1 61:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 5:1 6:1 17:1 29:1 39:1 41:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 17:1 22:1 36:1 41:1 48:1 66:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 14:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 11:1 15:1 24:1 38:1 42:1 52:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 17:1 19:1 39:1 40:1 55:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 6:1 16:1 25:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 14:1 25:1 38:1 42:1 48:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 14:1 22:1 36:1 41:1 51:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 15:1 20:1 37:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 7:1 14:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 8:1 15:1 19:1 39:1 41:1 51:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
+-1 1:1 8:1 16:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1
++1 4:1 8:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 2:1 6:1 18:1 19:1 39:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 16:1 20:1 37:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 52:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 6:1 18:1 22:1 36:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 3:1 6:1 18:1 22:1 36:1 40:1 51:1 61:1 67:1 72:1 75:1 76:1 80:1 83:1
+-1 2:1 11:1 18:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 17:1 21:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
++1 3:1 6:1 15:1 22:1 36:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 109:1
+-1 3:1 6:1 15:1 20:1 37:1 43:1 49:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 6:1 18:1 20:1 37:1 41:1 48:1 64:1 67:1 72:1 74:1 76:1 79:1 83:1
++1 5:1 7:1 14:1 19:1 39:1 41:1 56:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 4:1 6:1 16:1 27:1 35:1 40:1 49:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 3:1 6:1 16:1 19:1 39:1 42:1 50:1 64:1 67:1 72:1 74:1 77:1 80:1 83:1
+-1 4:1 6:1 15:1 20:1 37:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 16:1 29:1 39:1 42:1 52:1 64:1 68:1 73:1 74:1 76:1 78:1 90:1
+-1 5:1 6:1 15:1 22:1 36:1 43:1 57:1 64:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 11:1 17:1 19:1 39:1 41:1 55:1 64:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 5:1 7:1 14:1 22:1 36:1 40:1 56:1 63:1 67:1 73:1 75:1 76:1 80:1 83:1
++1 5:1 6:1 16:1 22:1 36:1 40:1 53:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 42:1 55:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 4:1 10:1 15:1 20:1 37:1 40:1 59:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 20:1 37:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 14:1 19:1 39:1 42:1 50:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 41:1 53:1 65:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 15:1 24:1 38:1 40:1 48:1 63:1 67:1 73:1 74:1 77:1 82:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 4:1 6:1 18:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 7:1 18:1 19:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 1:1 6:1 18:1 24:1 38:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 10:1 15:1 24:1 38:1 42:1 55:1 62:1 71:1 72:1 74:1 76:1 79:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 40:1 52:1 63:1 71:1 73:1 74:1 76:1 82:1
+-1 3:1 7:1 15:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
++1 4:1 6:1 18:1 29:1 39:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 11:1 14:1 19:1 39:1 41:1 55:1 66:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 15:1 22:1 36:1 40:1 49:1 61:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 2:1 18:1 22:1 36:1 41:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 20:1 37:1 41:1 53:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 6:1 17:1 22:1 36:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 3:1 6:1 16:1 19:1 39:1 44:1 52:1 66:1 67:1 72:1 74:1 76:1 81:1 83:1
+-1 3:1 6:1 16:1 22:1 36:1 41:1 57:1 66:1 67:1 73:1 74:1 76:1 81:1 83:1
++1 3:1 11:1 16:1 19:1 39:1 40:1 59:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
++1 2:1 6:1 15:1 19:1 39:1 42:1 51:1 64:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 16:1 22:1 36:1 41:1 48:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 7:1 17:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 5:1 18:1 22:1 36:1 40:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 15:1 20:1 37:1 42:1 55:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 17:1 20:1 37:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 41:1 49:1 64:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 5:1 8:1 17:1 20:1 37:1 44:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 3:1 6:1 16:1 19:1 39:1 40:1 51:1 63:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 5:1 7:1 14:1 24:1 38:1 41:1 47:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 8:1 15:1 29:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 1:1 6:1 17:1 22:1 36:1 42:1 56:1 62:1 67:1 73:1 74:1 76:1 81:1 83:1
+-1 2:1 6:1 15:1 26:1 35:1 40:1 56:1 63:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 14:1 21:1 35:1 42:1 55:1 65:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 17:1 22:1 36:1 40:1 57:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 16:1 27:1 35:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 103:1
+-1 1:1 6:1 17:1 19:1 39:1 42:1 55:1 64:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 2:1 8:1 17:1 19:1 39:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 82:1 87:1
++1 4:1 9:1 18:1 20:1 37:1 42:1 55:1 64:1 67:1 73:1 75:1 76:1 82:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 6:1 18:1 25:1 38:1 42:1 50:1 64:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 3:1 11:1 15:1 19:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 16:1 19:1 39:1 41:1 52:1 66:1 71:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 10:1 16:1 22:1 36:1 40:1 48:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
++1 4:1 8:1 15:1 20:1 37:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 4:1 6:1 15:1 22:1 36:1 41:1 54:1 66:1 67:1 73:1 74:1 76:1 79:1 83:1
++1 4:1 6:1 18:1 22:1 36:1 40:1 49:1 63:1 67:1 73:1 74:1 77:1 80:1 83:1
+-1 2:1 6:1 16:1 22:1 36:1 40:1 54:1 62:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 5:1 6:1 16:1 22:1 36:1 40:1 57:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
++1 5:1 10:1 14:1 32:1 39:1 40:1 51:1 63:1 67:1 73:1 74:1 76:1 82:1 83:1
+-1 5:1 6:1 14:1 19:1 39:1 40:1 47:1 63:1 67:1 73:1 74:1 76:1 80:1 83:1
+-1 1:1 15:1 21:1 35:1 42:1 62:1 67:1 72:1 74:1 76:1 78:1 83:1
+-1 4:1 7:1 17:1 25:1 38:1 40:1 52:1 63:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 5:1 6:1 18:1 26:1 35:1 40:1 49:1 63:1 71:1 73:1 74:1 76:1 81:1 83:1
+-1 4:1 6:1 17:1 33:1 35:1 40:1 54:1 63:1 67:1 73:1 74:1 76:1 80:1
+-1 1:1 6:1 17:1 20:1 37:1 42:1 50:1 64:1 67:1 72:1 74:1 76:1 82:1 83:1
++1 4:1 8:1 14:1 29:1 39:1 40:1 50:1 63:1 67:1 73:1 74:1 76:1 82:1
+-1 5:1 11:1 17:1 27:1 35:1 40:1 49:1 61:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 2:1 6:1 15:1 22:1 36:1 40:1 54:1 61:1 67:1 72:1 74:1 76:1 80:1 83:1
+-1 3:1 6:1 14:1 22:1 36:1 41:1 55:1 66:1 71:1 72:1 74:1 76:1 80:1 83:1
+-1 4:1 7:1 17:1 22:1 36:1 41:1 50:1 62:1 67:1 73:1 74:1 76:1 82:1
+-1 1:1 6:1 18:1 21:1 35:1 42:1 52:1 62:1 67:1 73:1 74:1 76:1 78:1 83:1
+-1 1:1 6:1 18:1 22:1 36:1 42:1 48:1 65:1 71:1 73:1 74:1 76:1 80:1 83:1
+-1 4:1 10:1 15:1 24:1 38:1 41:1 52:1 66:1 67:1 72:1 74:1 76:1 82:1 83:1
\ No newline at end of file
diff --git a/src/test/scripts/functions/io/libsvm/in/transfusion_W2.libsvm.mtd b/src/test/scripts/functions/io/libsvm/in/transfusion_W2.libsvm.mtd
new file mode 100644
index 0000000..79c852c
--- /dev/null
+++ b/src/test/scripts/functions/io/libsvm/in/transfusion_W2.libsvm.mtd
@@ -0,0 +1,11 @@
+{
+ "data_type": "matrix",
+ "value_type": "double",
+ "rows": 1605,
+ "cols": 124,
+ "sep": " ",
+ "indSep": ":",
+ "format": "libsvm",
+ "author": "Saeed Fathollahzadeh",
+ "created": "2021-04-14 17:27:25 CEST"
+}
diff --git a/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml b/src/test/scripts/functions/io/libsvm/libsvm_verify.R
similarity index 54%
copy from src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml
copy to src/test/scripts/functions/io/libsvm/libsvm_verify.R
index ec8d1cc..41f306c 100644
--- a/src/test/scripts/functions/io/libsvm/WriteLIBSVMTest_1.dml
+++ b/src/test/scripts/functions/io/libsvm/libsvm_verify.R
@@ -7,9 +7,9 @@
# 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
@@ -19,12 +19,40 @@
#
#-------------------------------------------------------------
-# DML script that tests read/write libsvm
+args <- commandArgs(TRUE)
+
+library("Matrix")
+options(digits=22)
+
+filename <- args[1]
+dimensionality <- as.integer(args[2])
+sep <- args[3]
+indSep <- args[4]
-A = read($1, format="libsvm", rows=7, cols=5);
-A = A + 1;
-x = sum(A);
-write(x, $2);
+if(sep == 'NULL'){
+ sep=" "
+}
-write(A, $3, format="libsvm", sparse=$4);
+content = readLines( filename )
+num_lines = length( content )
+A = matrix( 0, num_lines, dimensionality + 1 )
+# loop over lines
+for ( i in 1:num_lines ) {
+ # split by sep
+ line = as.vector( strsplit( content[i], sep )[[1]])
+ # save label
+ A[i,1] = as.numeric( line[[1]] )
+ # loop over values
+ for ( j in 2:length( line )) {
+ # split by colon
+ index_value = strsplit( line[j], indSep )[[1]]
+ index = as.numeric( index_value[1] ) + 1
+ value = as.numeric( index_value[2] )
+ A[i, index] = value
+ }
+ }
+A[is.na(A)] = 0
+A <- A+1
+x = sum(A)
+write(x, args[5])
\ No newline at end of file