aboutsummaryrefslogtreecommitdiffhomepage
path: root/tensorflow/contrib/lite
diff options
context:
space:
mode:
authorGravatar A. Unique TensorFlower <gardener@tensorflow.org>2018-02-27 15:19:47 -0800
committerGravatar TensorFlower Gardener <gardener@tensorflow.org>2018-02-27 15:23:57 -0800
commit2c25f08b6f97155bd5ce95aada5a3cc9b916176f (patch)
tree115fe5a596c07d49c9199fc47c28be5f75d3ad26 /tensorflow/contrib/lite
parente101ce9c1c8399fecd6679293d8cb2065ce8d47f (diff)
Implement support for unpartitioning tf.nn.embedding_lookup into a single gather.
PiperOrigin-RevId: 187241089
Diffstat (limited to 'tensorflow/contrib/lite')
-rw-r--r--tensorflow/contrib/lite/toco/BUILD1
-rw-r--r--tensorflow/contrib/lite/toco/graph_transformations/graph_transformations.h1
-rw-r--r--tensorflow/contrib/lite/toco/graph_transformations/propagate_fixed_sizes.cc6
-rw-r--r--tensorflow/contrib/lite/toco/graph_transformations/remove_trivial_passthrough.cc4
-rw-r--r--tensorflow/contrib/lite/toco/graph_transformations/unpartition_embedding_lookup.cc237
-rw-r--r--tensorflow/contrib/lite/toco/import_tensorflow.cc41
-rw-r--r--tensorflow/contrib/lite/toco/model.h26
-rw-r--r--tensorflow/contrib/lite/toco/toco_tooling.cc1
-rw-r--r--tensorflow/contrib/lite/toco/tooling_util.cc11
-rw-r--r--tensorflow/contrib/lite/toco/tooling_util.h6
10 files changed, 331 insertions, 3 deletions
diff --git a/tensorflow/contrib/lite/toco/BUILD b/tensorflow/contrib/lite/toco/BUILD
index 17407f3db2..845bc0460f 100644
--- a/tensorflow/contrib/lite/toco/BUILD
+++ b/tensorflow/contrib/lite/toco/BUILD
@@ -240,6 +240,7 @@ cc_library(
"graph_transformations/resolve_tensorflow_tile.cc",
"graph_transformations/resolve_transpose_attributes.cc",
"graph_transformations/unfuse_activation_functions.cc",
+ "graph_transformations/unpartition_embedding_lookup.cc",
"graph_transformations/unroll_batch_matmul.cc",
],
hdrs = [
diff --git a/tensorflow/contrib/lite/toco/graph_transformations/graph_transformations.h b/tensorflow/contrib/lite/toco/graph_transformations/graph_transformations.h
index f2c81ebc81..f0739990ad 100644
--- a/tensorflow/contrib/lite/toco/graph_transformations/graph_transformations.h
+++ b/tensorflow/contrib/lite/toco/graph_transformations/graph_transformations.h
@@ -177,6 +177,7 @@ DECLARE_GRAPH_TRANSFORMATION(ResolveConstantStridedSlice)
DECLARE_GRAPH_TRANSFORMATION(ResolveConstantFill)
DECLARE_GRAPH_TRANSFORMATION(ResolveMultiplyByZero)
DECLARE_GRAPH_TRANSFORMATION(Dequantize)
+DECLARE_GRAPH_TRANSFORMATION(UnpartitionEmbeddingLookup)
class ResolveReshapeAttributes : public GraphTransformation {
public:
diff --git a/tensorflow/contrib/lite/toco/graph_transformations/propagate_fixed_sizes.cc b/tensorflow/contrib/lite/toco/graph_transformations/propagate_fixed_sizes.cc
index 0e2e5ecf30..fc26f997a6 100644
--- a/tensorflow/contrib/lite/toco/graph_transformations/propagate_fixed_sizes.cc
+++ b/tensorflow/contrib/lite/toco/graph_transformations/propagate_fixed_sizes.cc
@@ -1542,6 +1542,12 @@ bool PropagateFixedSizes::Run(Model* model, std::size_t op_index) {
case OperatorType::kTranspose:
ProcessTransposeOperator(model, static_cast<TransposeOperator*>(op));
break;
+ case OperatorType::kDynamicPartition:
+ case OperatorType::kDynamicStitch:
+ // DynamicPartition/DynamicStitch are currently only supported for
+ // transforms that remove them, so we avoid propagating shapes through
+ // them and let things settle once they've been removed.
+ break;
default:
// Unimplemented, another graph transformation should drop it.
LOG(FATAL) << "Unhandled operator type " << OperatorTypeName(op->type);
diff --git a/tensorflow/contrib/lite/toco/graph_transformations/remove_trivial_passthrough.cc b/tensorflow/contrib/lite/toco/graph_transformations/remove_trivial_passthrough.cc
index 587f171bbf..aa93ace03a 100644
--- a/tensorflow/contrib/lite/toco/graph_transformations/remove_trivial_passthrough.cc
+++ b/tensorflow/contrib/lite/toco/graph_transformations/remove_trivial_passthrough.cc
@@ -60,7 +60,9 @@ bool RemoveTrivialPassthroughOp(GraphTransformation* transformation,
for (int i = 0; i < passthru_op->inputs.size(); i++) {
if (!model->GetArray(passthru_op->inputs[i]).buffer) {
count_nonconstant_input_arrays++;
- main_input_array_index = i;
+ if (count_nonconstant_input_arrays == 1) {
+ main_input_array_index = i;
+ }
}
}
diff --git a/tensorflow/contrib/lite/toco/graph_transformations/unpartition_embedding_lookup.cc b/tensorflow/contrib/lite/toco/graph_transformations/unpartition_embedding_lookup.cc
new file mode 100644
index 0000000000..419fb9a799
--- /dev/null
+++ b/tensorflow/contrib/lite/toco/graph_transformations/unpartition_embedding_lookup.cc
@@ -0,0 +1,237 @@
+/* Copyright 2018 The TensorFlow Authors. All Rights Reserved.
+
+Licensed 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.
+==============================================================================*/
+#include <memory>
+#include <string>
+#include <vector>
+
+#include "tensorflow/contrib/lite/toco/graph_transformations/graph_transformations.h"
+#include "tensorflow/contrib/lite/toco/model.h"
+#include "tensorflow/contrib/lite/toco/tooling_util.h"
+
+namespace toco {
+
+bool UnpartitionEmbeddingLookup::Run(Model* model, std::size_t op_index) {
+ // Collapses a partitioned tf.nn.embedding_lookup back into a single Gather.
+ // https://www.tensorflow.org/api_docs/python/tf/nn/embedding_lookup
+ // This transform attempts to identify the len(params) > 1 case and collapse
+ // it to the len(params) = 1 case by concatenating the original params and
+ // reversing the partitioning.
+ //
+ // If len(params) to the tf.nn.embedding_lookup == 1, the whole op becomes
+ // simply a gather:
+ // https://github.com/tensorflow/tensorflow/blob/r1.5/tensorflow/python/ops/embedding_ops.py#L150
+ //
+ // Notes on this implementation:
+ // - only supports partition_strategy='mod'
+ //
+ // A rough graph of a partitioned embedding_lookup looks like:
+ // (ids)--+-->FloorDiv--+-->DynamicPartition-->[[Gather]]--\
+ // \-->FloorMod--/ |
+ // V |
+ // Range-->DynamicPartition-------->DynamicStitch<---------/
+ // (const) V
+ // (embeddings)
+
+ // First look for the final DynamicStitch.
+ auto op_it = model->operators.begin() + op_index;
+ if (op_it->get()->type != OperatorType::kDynamicStitch) {
+ return false;
+ }
+ auto* stitch_op = static_cast<DynamicStitchOperator*>(op_it->get());
+
+ // Split up the DynamicStitch inputs into the indices and data.
+ std::vector<string> stitch_indices_inputs;
+ std::vector<string> stitch_data_inputs;
+ for (size_t i = 0; i < stitch_op->num_partitions; ++i) {
+ stitch_indices_inputs.push_back(stitch_op->inputs[i]);
+ }
+ for (size_t i = stitch_op->num_partitions; i < stitch_op->num_partitions * 2;
+ ++i) {
+ stitch_data_inputs.push_back(stitch_op->inputs[i]);
+ }
+
+ // Validate all indices come from the same DynamicPartition.
+ DynamicPartitionOperator* indices_partition_op = nullptr;
+ for (const string& indices_partition_output_name : stitch_indices_inputs) {
+ auto* op = GetOpWithOutput(*model, indices_partition_output_name);
+ CHECK(op) << "Source of " << indices_partition_output_name << " not found";
+ if (op->type != OperatorType::kDynamicPartition) {
+ AddMessageF(
+ "Skipping because indices input %s into "
+ "%s is unexpected",
+ LogName(*op), LogName(*stitch_op));
+ return false;
+ }
+ if (!indices_partition_op) {
+ indices_partition_op = static_cast<DynamicPartitionOperator*>(op);
+ } else {
+ // Ensure this is the same op as previous ones.
+ if (op != indices_partition_op) {
+ AddMessageF(
+ "Skipping because indices input %s into "
+ "%s is from a different source op than others",
+ LogName(*op), LogName(*stitch_op));
+ return false;
+ }
+ }
+ }
+ CHECK(indices_partition_op) << "No indices inputs";
+
+ // The data for the indices must be a constant range of the array shape.
+ if (!IsConstantParameterArray(*model, indices_partition_op->inputs[0])) {
+ AddMessageF("Skipping because indices partition data is non-constant");
+ return false;
+ }
+ auto& indices_data_array = model->GetArray(indices_partition_op->inputs[0]);
+ if (indices_data_array.data_type == ArrayDataType::kNone) {
+ // Yield until data types are propagated.
+ return false;
+ }
+ CHECK(indices_data_array.data_type == ArrayDataType::kInt32)
+ << "Indices partition inputs must be int32";
+ const auto& indices_data_buffer =
+ indices_data_array.GetBuffer<ArrayDataType::kInt32>().data;
+ for (size_t i = 0; i < indices_data_buffer.size(); ++i) {
+ CHECK_EQ(indices_data_buffer[i], i) << "Indices range must be identity";
+ }
+
+ // Find all of the gathers used for the data inputs.
+ std::vector<GatherOperator*> gather_ops;
+ for (const string& gather_output_name : stitch_data_inputs) {
+ auto* op = GetOpWithOutput(*model, gather_output_name);
+ CHECK(op) << "Source of " << gather_output_name << " not found";
+ if (op->type != OperatorType::kGather) {
+ AddMessageF(
+ "Skipping because data input %s into %s "
+ "is unexpected",
+ LogName(*op), LogName(*stitch_op));
+ return false;
+ }
+ gather_ops.push_back(static_cast<GatherOperator*>(op));
+ }
+
+ // Validate all gathers come from the same DynamicPartition.
+ DynamicPartitionOperator* data_partition_op = nullptr;
+ for (auto* gather_op : gather_ops) {
+ auto* op = GetOpWithOutput(*model, gather_op->inputs[1]);
+ CHECK(op) << "Source of " << gather_op->inputs[1] << " not found";
+ if (op->type != OperatorType::kDynamicPartition) {
+ AddMessageF(
+ "Skipping because data input %s into "
+ "%s is unexpected",
+ LogName(*op), LogName(*gather_op));
+ return false;
+ }
+ if (!data_partition_op) {
+ data_partition_op = static_cast<DynamicPartitionOperator*>(op);
+ } else {
+ // Ensure this is the same op as previous ones.
+ if (op != data_partition_op) {
+ AddMessageF(
+ "Skipping because data input %s into "
+ "%s is from a different source op than others",
+ LogName(*op), LogName(*gather_op));
+ return false;
+ }
+ }
+ }
+ CHECK(data_partition_op) << "No data inputs";
+
+ // Validate the partition ops have the same sizes.
+ CHECK_EQ(indices_partition_op->num_partitions,
+ data_partition_op->num_partitions)
+ << "Indices and data partition ops have differing dimensions";
+ int num_partitions = indices_partition_op->num_partitions;
+
+ // Partition strategy of 'mod' gives us a FloorMod and FloorDiv.
+ // The gather partition uses the FloorDiv as the data and FloorMod as the
+ // partitions and the indices use the FloorMod as their partitions.
+ Operator* div_op = GetOpWithOutput(*model, data_partition_op->inputs[0]);
+ Operator* mod_op = GetOpWithOutput(*model, data_partition_op->inputs[1]);
+ CHECK(div_op && div_op->type == OperatorType::kFloorDiv)
+ << "Unsupported partition strategy";
+ CHECK(mod_op && mod_op->type == OperatorType::kFloorMod)
+ << "Unsupported partition strategy";
+ CHECK_EQ(mod_op, GetOpWithOutput(*model, indices_partition_op->inputs[1]))
+ << "Indices and data parition ops require the same partition strategy "
+ "and inputs";
+
+ // Glob together all of the gather data. This is not yet in the correct order.
+ auto* gather_params_concat_op = new ConcatenationOperator;
+ for (const auto& gather_op : gather_ops) {
+ gather_params_concat_op->inputs.push_back(gather_op->inputs[0]);
+ }
+ gather_params_concat_op->outputs.push_back(
+ AvailableArrayName(*model, gather_ops[0]->inputs[0] + "_unpartitioned"));
+ op_it = model->operators.emplace(op_it, gather_params_concat_op) + 1;
+ model->GetOrCreateArray(gather_params_concat_op->outputs[0]);
+
+ // Permute the gather params to undo the partitioning that was originally
+ // done.
+ auto* gather_params_permute_op = new GatherOperator;
+ gather_params_permute_op->inputs.push_back(
+ gather_params_concat_op->outputs[0]);
+ gather_params_permute_op->inputs.push_back(
+ AvailableArrayName(*model, gather_ops[0]->inputs[0] + "_permuted/perm"));
+ gather_params_permute_op->outputs.push_back(
+ AvailableArrayName(*model, gather_ops[0]->inputs[0] + "_permuted"));
+ op_it = model->operators.emplace(op_it, gather_params_permute_op) + 1;
+ model->GetOrCreateArray(gather_params_permute_op->outputs[0]);
+ const auto& partition_array = model->GetArray(gather_ops[0]->inputs[0]);
+ const auto& partition_array_dims = partition_array.shape().dims();
+ auto& perm_array =
+ model->GetOrCreateArray(gather_params_permute_op->inputs[1]);
+ perm_array.data_type = ArrayDataType::kInt32;
+ perm_array.mutable_shape()->ReplaceDims(
+ {num_partitions * partition_array_dims[0]});
+ auto& perm_data = perm_array.GetMutableBuffer<ArrayDataType::kInt32>().data;
+ perm_data.resize(RequiredBufferSizeForShape(perm_array.shape()));
+ // NOTE: this is what relies on the partition_strategy.
+ for (int i = 0; i < num_partitions * partition_array_dims[0]; ++i) {
+ int p = i % num_partitions;
+ perm_data[i] = p * partition_array_dims[0] + i / num_partitions;
+ }
+
+ // Insert the new unpartitioned gather op.
+ auto* merged_gather_op = new GatherOperator;
+ merged_gather_op->inputs = {gather_params_permute_op->outputs[0],
+ mod_op->inputs[0]};
+ merged_gather_op->outputs = {stitch_op->outputs[0]};
+ model->operators.emplace(op_it, merged_gather_op);
+
+ AddMessageF(
+ "Replacing suspected partitioned tf.nn.embedding_lookup (starting at %s "
+ "+ %s and ending at %s) with a single unpartitioned gather %s",
+ LogName(*div_op), LogName(*mod_op), LogName(*stitch_op),
+ LogName(*merged_gather_op));
+
+ // Ensure the stitch output array is dead, as we don't want whatever was in it
+ // previously now that we've redefined it. It'll be recreated when needed.
+ model->EraseArray(stitch_op->outputs[0]);
+ model->GetOrCreateArray(merged_gather_op->outputs[0]);
+
+ // Erase all the original ops.
+ DeleteOpAndArraysIfUnused(model, div_op);
+ DeleteOpAndArraysIfUnused(model, mod_op);
+ for (auto* gather_op : gather_ops) {
+ DeleteOpAndArraysIfUnused(model, gather_op);
+ }
+ DeleteOpAndArraysIfUnused(model, indices_partition_op);
+ DeleteOpAndArraysIfUnused(model, data_partition_op);
+ DeleteOpAndArraysIfUnused(model, stitch_op);
+ return true;
+}
+
+} // namespace toco
diff --git a/tensorflow/contrib/lite/toco/import_tensorflow.cc b/tensorflow/contrib/lite/toco/import_tensorflow.cc
index 52a0512e23..41abca864d 100644
--- a/tensorflow/contrib/lite/toco/import_tensorflow.cc
+++ b/tensorflow/contrib/lite/toco/import_tensorflow.cc
@@ -1896,6 +1896,42 @@ void ConvertTopKV2Operator(const NodeDef& node,
op->outputs.push_back(node.name() + ":1");
model->operators.emplace_back(op.release());
}
+
+void ConvertDynamicPartitionOperator(
+ const NodeDef& node, const TensorFlowImportFlags& tf_import_flags,
+ Model* model) {
+ auto op = absl::make_unique<DynamicPartitionOperator>();
+ CHECK(HasAttr(node, "num_partitions"));
+ op->num_partitions = GetIntAttr(node, "num_partitions");
+ CheckInputsCount(node, tf_import_flags, 2);
+ op->inputs.push_back(node.input(0));
+ op->inputs.push_back(node.input(1));
+ CHECK_GT(op->num_partitions, 1);
+ op->outputs.push_back(node.name()); // Implicit :0.
+ for (int i = 1; i < op->num_partitions; ++i) {
+ op->outputs.push_back(node.name() + ":" + std::to_string(i));
+ }
+ model->operators.emplace_back(op.release());
+}
+
+void ConvertDynamicStitchOperator(const NodeDef& node,
+ const TensorFlowImportFlags& tf_import_flags,
+ Model* model) {
+ // The parallel and non-parallel variants are the same besides whether they
+ // have a parallel loop; there are no behavioral differences.
+ CHECK(node.op() == "DynamicStitch" || node.op() == "ParallelDynamicStitch");
+ auto op = absl::make_unique<DynamicStitchOperator>();
+ CHECK(HasAttr(node, "N"));
+ op->num_partitions = GetIntAttr(node, "N");
+ // Expect all ID partitions + all value partitions.
+ CheckInputsCount(node, tf_import_flags, op->num_partitions * 2);
+ for (int i = 0; i < op->num_partitions * 2; ++i) {
+ op->inputs.push_back(node.input(i));
+ }
+ op->outputs.push_back(node.name());
+ model->operators.emplace_back(op.release());
+}
+
} // namespace
std::unique_ptr<Model> ImportTensorFlowGraphDef(
@@ -2081,6 +2117,11 @@ std::unique_ptr<Model> ImportTensorFlowGraphDef(
ConvertExpOperator(node, tf_import_flags, model);
} else if (node.op() == "TopK" || node.op() == "TopKV2") {
ConvertTopKV2Operator(node, tf_import_flags, model);
+ } else if (node.op() == "DynamicPartition") {
+ ConvertDynamicPartitionOperator(node, tf_import_flags, model);
+ } else if (node.op() == "DynamicStitch" ||
+ node.op() == "ParallelDynamicStitch") {
+ ConvertDynamicStitchOperator(node, tf_import_flags, model);
} else {
ConvertUnsupportedOperator(node, tf_import_flags, model);
}
diff --git a/tensorflow/contrib/lite/toco/model.h b/tensorflow/contrib/lite/toco/model.h
index d5df0fb951..ed0dedc003 100644
--- a/tensorflow/contrib/lite/toco/model.h
+++ b/tensorflow/contrib/lite/toco/model.h
@@ -115,6 +115,8 @@ enum class OperatorType {
kTensorFlowTile,
kTranspose,
kTopK_V2,
+ kDynamicPartition,
+ kDynamicStitch,
// An unsupported TF operation. It's only needed to be able to represent TF
// graph internally and is expected to be dropped by graph transformations.
kTensorFlowUnsupported,
@@ -1414,6 +1416,30 @@ struct TopKV2Operator : Operator {
TopKV2Operator() : Operator(OperatorType::kTopK_V2) {}
};
+// DynamicPartition operator:
+//
+// Inputs:
+// inputs[0]: required: data.
+// inputs[1]: required: partitions.
+//
+// TensorFlow equivalent: DynamicPartition
+struct DynamicPartitionOperator : Operator {
+ DynamicPartitionOperator() : Operator(OperatorType::kDynamicPartition) {}
+ int num_partitions;
+};
+
+// DynamicStitch operator:
+//
+// Inputs:
+// inputs[0,N): required: indices.
+// inputs[N,2N): required: data.
+//
+// TensorFlow equivalent: DynamicStitch/ParallelDynamicStitch
+struct DynamicStitchOperator : Operator {
+ DynamicStitchOperator() : Operator(OperatorType::kDynamicStitch) {}
+ int num_partitions;
+};
+
// Alloc's are used for transient arrays only. An Alloc specifies which interval
// of the "transient_data" workspace buffer passed to inference functions, is to
// be used for the transient array at hand. The 'start' and 'end' values are
diff --git a/tensorflow/contrib/lite/toco/toco_tooling.cc b/tensorflow/contrib/lite/toco/toco_tooling.cc
index a09a3c4ef5..42e0a89017 100644
--- a/tensorflow/contrib/lite/toco/toco_tooling.cc
+++ b/tensorflow/contrib/lite/toco/toco_tooling.cc
@@ -102,6 +102,7 @@ void MakeGeneralGraphTransformationsSet(
transformations->Add(new ResolveConstantShapeOrRank);
transformations->Add(new MakeInitialDequantizeOperator);
transformations->Add(new ResolveConstantFakeQuant);
+ transformations->Add(new UnpartitionEmbeddingLookup);
}
bool SupportsQuantization(FileFormat format) {
diff --git a/tensorflow/contrib/lite/toco/tooling_util.cc b/tensorflow/contrib/lite/toco/tooling_util.cc
index d23b3737fc..f92e10752d 100644
--- a/tensorflow/contrib/lite/toco/tooling_util.cc
+++ b/tensorflow/contrib/lite/toco/tooling_util.cc
@@ -159,6 +159,15 @@ bool DeleteArrayIfUsedOnce(const string& array_name, Model* model) {
return false;
}
+void DeleteOpAndArraysIfUnused(Model* model, Operator* op) {
+ for (const string& array_name : op->inputs) {
+ DeleteArrayIfUsedOnce(array_name, model);
+ }
+ auto op_it = FindOp(*model, op);
+ CHECK(op_it != model->operators.end());
+ model->operators.erase(op_it);
+}
+
std::vector<std::unique_ptr<Operator>>::const_iterator FindOpWithOutput(
const Model& model, const string& array_name) {
for (auto it = model.operators.begin(); it != model.operators.end(); ++it) {
@@ -347,6 +356,8 @@ const char* OperatorTypeName(OperatorType type) {
HANDLE_OPERATORTYPENAME_CASE(TopK_V2)
HANDLE_OPERATORTYPENAME_CASE(TensorFlowUnsupported)
HANDLE_OPERATORTYPENAME_CASE(Exp)
+ HANDLE_OPERATORTYPENAME_CASE(DynamicPartition)
+ HANDLE_OPERATORTYPENAME_CASE(DynamicStitch)
default:
LOG(FATAL) << "Unhandled op type";
#undef HANDLE_OPERATORTYPENAME_CASE
diff --git a/tensorflow/contrib/lite/toco/tooling_util.h b/tensorflow/contrib/lite/toco/tooling_util.h
index 11208ed667..01917b29de 100644
--- a/tensorflow/contrib/lite/toco/tooling_util.h
+++ b/tensorflow/contrib/lite/toco/tooling_util.h
@@ -64,6 +64,10 @@ int CountOpsWithInput(const Model& model, const string& array_name);
bool DeleteArrayIfUnused(const string& array_name, Model* model);
bool DeleteArrayIfUsedOnce(const string& array_name, Model* model);
+// Deletes the op and any of its input and output arrays if they are unused
+// after the op has been deleted.
+void DeleteOpAndArraysIfUnused(Model* model, Operator* op);
+
std::vector<std::unique_ptr<Operator>>::const_iterator FindOpWithOutput(
const Model& model, const string& array_name);
Operator* GetOpWithOutput(const Model& model, const string& array_name);
@@ -71,8 +75,6 @@ Operator* GetOpWithOutput(const Model& model, const string& array_name);
std::vector<std::unique_ptr<Operator>>::iterator FindOpWithOutput(
Model& model, const string& array_name);
-Operator* GetOpWithOutput(const Model& model, const string& array_name);
-
std::vector<std::unique_ptr<Operator>>::const_iterator FindOpWithInput(
const Model& model, const string& array_name);