Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -326,6 +326,13 @@ public double normL2Square() {
return d;
}

/**
* If the size of sparse vector is not given, it will be treated as
* a sparse vector with infinite size.
* Extract parts of a sparse vector and return the extracted parts
* in a new sparse vector. If the given indices exceed the
* maximum length of the vector, it will throw exception.
*/
@Override
public SparseVector slice(int[] indices) {
SparseVector sliced = new SparseVector(indices.length);
Expand All @@ -334,6 +341,9 @@ public SparseVector slice(int[] indices) {
sliced.values = new double[indices.length];

for (int i = 0; i < indices.length; i++) {
if (this.n >= 0 && indices[i] >= this.n) {
throw new IllegalArgumentException("Index is larger than vector size.");
}
int pos = Arrays.binarySearch(this.indices, indices[i]);
if (pos >= 0) {
sliced.indices[nnz] = i;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.flink.ml.operator.common.dataproc.vector;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.ml.api.misc.param.Params;
import org.apache.flink.ml.common.linalg.DenseVector;
import org.apache.flink.ml.common.linalg.SparseVector;
import org.apache.flink.ml.common.linalg.Vector;
import org.apache.flink.ml.common.mapper.Mapper;
import org.apache.flink.ml.common.utils.OutputColsHelper;
import org.apache.flink.ml.common.utils.TableUtil;
import org.apache.flink.ml.params.dataproc.vector.VectorToColumnsParams;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.types.Row;
import org.apache.flink.util.Preconditions;

import java.util.Arrays;

/**
* This mapper maps vector to table columns, and the table is created with the first
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
* This mapper maps vector to table columns, and the table is created with the first
* This is a data preprocessing function that transforms {@link Vector}s into {@link Table} columns.
*
* <p>Table is created with the first colSize value of the vector.
*
* <p>For sparse vector without given size, it will be treated as vector with infinite size.
* ...

* colSize value of the vector.
* For sparse vector without given size, it will be treated as vector with infinite size.
* If the colSize is larger than the vector size, we'll throw exception;
* If the colSize is not larger than the vector size, we'll select the first
* colSize value of the vector.
*/
public class VectorToColumnsMapper extends Mapper {
private int colSize;
private int idx;
private OutputColsHelper outputColsHelper;

public VectorToColumnsMapper(TableSchema dataSchema, Params params) {
Copy link

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you let a java doc with description which parameters should set successful creating new object, please?

super(dataSchema, params);
String selectedColName = this.params.get(VectorToColumnsParams.SELECTED_COL);
idx = TableUtil.findColIndex(dataSchema.getFieldNames(), selectedColName);
Preconditions.checkArgument(idx >= 0, "Can not find column: " + selectedColName);
String[] outputColNames = this.params.get(VectorToColumnsParams.OUTPUT_COLS);
Preconditions.checkArgument(null != outputColNames,
"VectorToTable: outputColNames must set.");
this.colSize = outputColNames.length;
TypeInformation[] types = new TypeInformation[colSize];
Arrays.fill(types, Types.DOUBLE);
this.outputColsHelper = new OutputColsHelper(dataSchema, outputColNames, types,
this.params.get(VectorToColumnsParams.RESERVED_COLS));
Comment on lines +54 to +63
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's refactor this part out as a private static function: constructOutputColsHelper ? I can see @ex00 's concerns that the constructor is way too complex. In principle the constructor should be simpler and easy to understand. One example is to have this as:

public VectorToColumnsMapper(TableSchema dataSchema, Params params) {
  this(dataSchema, params, constructOutputColsHelper(dataSchema, params));
}

public VectorToColumnsMapper(TableSchema dataSchema, Params params, OutputColsHelper outputColsHelper) {
  super(dataSchema, params);
  this.outputColsHelper = outputColsHelper;
}

}

@Override
public Row map(Row row) {
Row result = new Row(colSize);
Object obj = row.getField(idx);
if (null == obj) {
for (int i = 0; i < colSize; i++) {
result.setField(i, null);
}
return outputColsHelper.getResultRow(row, result);
}

Vector vec = (Vector) obj;

int vectorSize = vec.size();
if (vectorSize >= 0 && colSize > vectorSize) {
throw new RuntimeException("colSize is larger than vector size! colSize: "
+ colSize + ", vectorSize: " + vectorSize);
}
if (vec instanceof SparseVector) {
for (int i = 0; i < colSize; ++i) {
result.setField(i, 0.0);
}
SparseVector sparseVector = (SparseVector) vec;
int nnz = sparseVector.numberOfValues();
int[] indices = sparseVector.getIndices();
double[] values = sparseVector.getValues();
for (int i = 0; i < nnz; ++i) {
if (indices[i] < colSize) {
result.setField(indices[i], values[i]);
} else {
break;
}
}
} else {
DenseVector denseVector = (DenseVector) vec;
for (int i = 0; i < colSize; ++i) {
result.setField(i, denseVector.get(i));
}
}
return outputColsHelper.getResultRow(row, result);
}

@Override
public TableSchema getOutputSchema() {
return outputColsHelper.getResultSchema();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.flink.ml.params.dataproc.vector;

import org.apache.flink.ml.params.shared.colname.HasOutputCols;
import org.apache.flink.ml.params.shared.colname.HasReservedCols;
import org.apache.flink.ml.params.shared.colname.HasSelectedCol;

/**
* parameters of vector to columns.
*/
public interface VectorToColumnsParams<T> extends
HasSelectedCol<T>,
HasOutputCols<T>,
HasReservedCols<T> {
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.flink.ml.operator.common.dataproc.vector;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.ml.api.misc.param.Params;
import org.apache.flink.ml.common.linalg.DenseVector;
import org.apache.flink.ml.common.linalg.SparseVector;
import org.apache.flink.ml.params.dataproc.vector.VectorToColumnsParams;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.types.Row;

import org.junit.Test;

import static org.junit.Assert.assertEquals;

/**
* Unit test for VectorToColumnsMapper.
*/
public class VectorToColumnsMapperTest {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Test case looks good! thanks for refining @xuyang1706

@Test
public void testDenseReversed() throws Exception {
TableSchema schema = new TableSchema(new String[]{"vec"}, new TypeInformation<?>[]{Types.STRING});

Params params = new Params()
.set(VectorToColumnsParams.SELECTED_COL, "vec")
.set(VectorToColumnsParams.OUTPUT_COLS, new String[]{"f0", "f1"});

VectorToColumnsMapper mapper = new VectorToColumnsMapper(schema, params);
Row row = mapper.map(Row.of(new DenseVector(new double[]{3.0, 4.0})));
assertEquals(row.getField(1), 3.0);
assertEquals(row.getField(2), 4.0);
assertEquals(mapper.getOutputSchema(), new TableSchema(new String[]{"vec", "f0", "f1"},
new TypeInformation<?>[]{Types.STRING, Types.DOUBLE, Types.DOUBLE}));

}

@Test
public void testDense() throws Exception {
TableSchema schema = new TableSchema(new String[]{"vec"}, new TypeInformation<?>[]{Types.STRING});

Params params = new Params()
.set(VectorToColumnsParams.SELECTED_COL, "vec")
.set(VectorToColumnsParams.RESERVED_COLS, new String[]{})
.set(VectorToColumnsParams.OUTPUT_COLS, new String[]{"f0", "f1"});

VectorToColumnsMapper mapper = new VectorToColumnsMapper(schema, params);

Row row = mapper.map(Row.of(new DenseVector(new double[]{3.0, 4.0})));
assertEquals(row.getField(0), 3.0);
assertEquals(row.getField(1), 4.0);
assertEquals(mapper.getOutputSchema(), new TableSchema(new String[]{"f0", "f1"},
new TypeInformation<?>[]{Types.DOUBLE, Types.DOUBLE}));
}

@Test
public void testSparse() throws Exception {
TableSchema schema = new TableSchema(new String[]{"vec"}, new TypeInformation<?>[]{Types.STRING});
Params params = new Params()
.set(VectorToColumnsParams.SELECTED_COL, "vec")
.set(VectorToColumnsParams.OUTPUT_COLS, new String[]{"f0", "f1", "f2"});

VectorToColumnsMapper mapper = new VectorToColumnsMapper(schema, params);

Row row = mapper.map(Row.of(new SparseVector(3, new int[]{1, 2}, new double[]{3.0, 4.0})));
assertEquals(row.getField(0), new SparseVector(3, new int[]{1, 2}, new double[]{3.0, 4.0}));
assertEquals(row.getField(1), 0.0);
assertEquals(row.getField(2), 3.0);
assertEquals(row.getField(3), 4.0);
assertEquals(mapper.getOutputSchema(), new TableSchema(new String[]{"vec", "f0", "f1", "f2"},
new TypeInformation<?>[]{Types.STRING, Types.DOUBLE, Types.DOUBLE, Types.DOUBLE}));
}

@Test
public void testNull() throws Exception {
TableSchema schema = new TableSchema(new String[]{"vec"}, new TypeInformation<?>[]{Types.STRING});

Params params = new Params()
.set(VectorToColumnsParams.SELECTED_COL, "vec")
.set(VectorToColumnsParams.RESERVED_COLS, new String[]{})
.set(VectorToColumnsParams.OUTPUT_COLS, new String[]{"f0", "f1"});

VectorToColumnsMapper mapper = new VectorToColumnsMapper(schema, params);

Row row = mapper.map(Row.of((Object) null));
assertEquals(row.getField(0), null);
assertEquals(row.getField(1), null);
assertEquals(mapper.getOutputSchema(), new TableSchema(new String[]{"f0", "f1"},
new TypeInformation<?>[]{Types.DOUBLE, Types.DOUBLE}));
}
}